From ee6dddee0b5fdc860db0539b68d7f21d9a7c5b64 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 19 Jan 2026 13:00:52 +0800 Subject: [PATCH 01/53] [improve][broker] Ensure metadata session state visibility and improve Unstable observability for ServiceUnitStateChannelImpl (#25132) (cherry picked from commit 2a29be0d04f293c767ccf91dde807fd036d2ca95) (cherry picked from commit 85dc7585f5529352352f8057dd3e3813e6617916) --- .../channel/ServiceUnitStateChannelImpl.java | 20 ++++- .../channel/ServiceUnitStateChannelTest.java | 77 +++++++++++++++++++ 2 files changed, 94 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 8e992095c5222..39f918ba75dd5 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 @@ -127,8 +127,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private ServiceUnitStateTableView tableview; private ScheduledFuture monitorTask; - private SessionEvent lastMetadataSessionEvent = SessionReestablished; - private long lastMetadataSessionEventTimestamp = 0; + private volatile SessionEvent lastMetadataSessionEvent = SessionReestablished; + private volatile long lastMetadataSessionEventTimestamp = 0; private long inFlightStateWaitingTimeInMillis; private long ownershipMonitorDelayTimeInSecs; @@ -1703,7 +1703,9 @@ protected void monitorOwnerships(List brokers) { var metadataState = getMetadataState(); if (metadataState != Stable) { - log.warn("metadata state:{} is not Stable. Skipping ownership monitor.", metadataState); + log.warn("metadata state:{} is not Stable. Skipping ownership monitor. lastMetadataSessionEvent:{}," + + " lastMetadataSessionEventTimestamp:{}", + metadataState, lastMetadataSessionEvent, lastMetadataSessionEventTimestamp); return; } @@ -1972,9 +1974,21 @@ public List getMetrics() { var metric = Metrics.create(dimensions); + var metadataState = getMetadataState(); + long now = System.currentTimeMillis(); + long lastSessionEventAgeSeconds = + lastMetadataSessionEventTimestamp > 0 + ? MILLISECONDS.toSeconds(now - lastMetadataSessionEventTimestamp) + : -1; + metric.put("brk_sunit_state_chn_orphan_su_cleanup_ops_total", totalOrphanServiceUnitCleanupCnt); metric.put("brk_sunit_state_chn_su_tombstone_cleanup_ops_total", totalServiceUnitTombstoneCleanupCnt); metric.put("brk_sunit_state_chn_owned_su_total", getTotalOwnedServiceUnitCnt()); + metric.put("brk_sunit_state_chn_metadata_state", metadataState.ordinal()); + metric.put("brk_sunit_state_chn_last_metadata_session_event_is_reestablished", + lastMetadataSessionEvent == SessionReestablished ? 1 : 0); + metric.put("brk_sunit_state_chn_last_metadata_session_event_timestamp_ms", lastMetadataSessionEventTimestamp); + metric.put("brk_sunit_state_chn_last_metadata_session_event_age_seconds", lastSessionEventAgeSeconds); metrics.add(metric); return metrics; 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 bbde38bfbeceb..5baf91f2d5d24 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 @@ -96,6 +96,7 @@ import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreTableView; @@ -731,6 +732,82 @@ public void handleMetadataSessionEventTest() throws IllegalAccessException { } + + @Test + public void metadataStateMetricsTest() throws IllegalAccessException { + ServiceUnitStateChannelImpl channel1 = (ServiceUnitStateChannelImpl) this.channel1; + + long now = System.currentTimeMillis(); + long oldTimestamp = now - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000) - 1; + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEvent", SessionReestablished, true); + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEventTimestamp", oldTimestamp, true); + long beforeMetricsCall = System.currentTimeMillis(); + var metrics = channel1.getMetrics(); + long afterMetricsCall = System.currentTimeMillis(); + assertEquals(0, getMetric(metrics, "brk_sunit_state_chn_metadata_state").intValue()); + assertEquals(1, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_is_reestablished") + .intValue()); + assertEquals(oldTimestamp, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_timestamp_ms") + .longValue()); + long ageSeconds = getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_age_seconds") + .longValue(); + long minAgeSeconds = TimeUnit.MILLISECONDS.toSeconds(beforeMetricsCall - oldTimestamp); + long maxAgeSeconds = TimeUnit.MILLISECONDS.toSeconds(afterMetricsCall - oldTimestamp); + assertTrue(ageSeconds >= minAgeSeconds && ageSeconds <= maxAgeSeconds, + "Unexpected age seconds: " + ageSeconds + ", expected within [" + minAgeSeconds + ", " + + maxAgeSeconds + "]"); + + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEvent", SessionReestablished, true); + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEventTimestamp", now, true); + metrics = channel1.getMetrics(); + assertEquals(1, getMetric(metrics, "brk_sunit_state_chn_metadata_state").intValue()); + assertEquals(1, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_is_reestablished") + .intValue()); + assertEquals(now, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_timestamp_ms") + .longValue()); + ageSeconds = getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_age_seconds") + .longValue(); + assertTrue(ageSeconds >= 0 && ageSeconds <= 1, "Unexpected age seconds: " + ageSeconds); + + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEvent", SessionLost, true); + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEventTimestamp", now, true); + metrics = channel1.getMetrics(); + assertEquals(2, getMetric(metrics, "brk_sunit_state_chn_metadata_state").intValue()); + assertEquals(0, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_is_reestablished") + .intValue()); + assertEquals(now, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_timestamp_ms") + .longValue()); + ageSeconds = getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_age_seconds") + .longValue(); + assertTrue(ageSeconds >= 0 && ageSeconds <= 1, "Unexpected age seconds: " + ageSeconds); + + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEvent", SessionReestablished, true); + FieldUtils.writeDeclaredField(channel1, "lastMetadataSessionEventTimestamp", 0L, true); + metrics = channel1.getMetrics(); + assertEquals(0, getMetric(metrics, "brk_sunit_state_chn_metadata_state").intValue()); + assertEquals(1, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_is_reestablished") + .intValue()); + assertEquals(0L, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_timestamp_ms") + .longValue()); + assertEquals(-1L, getMetric(metrics, "brk_sunit_state_chn_last_metadata_session_event_age_seconds") + .longValue()); + } + + private static Number getMetric(List metrics, String metricName) { + for (Metrics metric : metrics) { + Object value = metric.getMetrics().get(metricName); + if (value == null) { + continue; + } + if (!(value instanceof Number)) { + fail(metricName + " is not numeric: " + value); + } + return (Number) value; + } + fail("Missing " + metricName + " metric"); + return -1L; + } + @Test(priority = 8) public void handleBrokerCreationEventTest() throws IllegalAccessException { var cleanupJobs = getCleanupJobs(channel1); From edf83cc53c118dcb35e392623aaac785792c2cff Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 19 Jan 2026 13:00:52 +0800 Subject: [PATCH 02/53] [improve][broker] Ensure metadata session state visibility and improve Unstable observability for ServiceUnitStateChannelImpl (#25132) (cherry picked from commit 2a29be0d04f293c767ccf91dde807fd036d2ca95) (cherry picked from commit 85dc7585f5529352352f8057dd3e3813e6617916) From 1d793742a8234eb030462933f614ff137554dd0b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 21 Jan 2026 13:22:59 +0800 Subject: [PATCH 03/53] [improve][broker] Upgrade bookkeeper to 4.17.3 (#25166) (cherry picked from commit 45def39b12be0d760b700c2e5da085773e09ba3d) (cherry picked from commit 333110ab00494058dd88b9773d37ab4e78f92cc4) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 10 +++++----- pom.xml | 7 ++++++- .../apache/bookkeeper/client/PulsarMockBookKeeper.java | 3 ++- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index a2439e82ea261..e2a36d69914a0 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -540,7 +540,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index ac765378656bd..7af948c69be53 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -338,11 +338,11 @@ The Apache Software License, Version 2.0 - memory-0.8.3.jar - sketches-core-0.8.3.jar * Apache Commons - - commons-codec-1.18.0.jar - - commons-io-2.19.0.jar - - commons-lang3-3.18.0.jar - - commons-text-1.13.1.jar - - commons-compress-1.27.1.jar + - commons-codec-1.20.0.jar + - commons-io-2.21.0.jar + - commons-lang3-3.19.0.jar + - commons-text-1.14.0.jar + - commons-compress-1.28.0.jar * Netty - netty-buffer-4.1.130.Final.jar - netty-codec-4.1.130.Final.jar diff --git a/pom.xml b/pom.xml index e26c7f489ef59..11166d5e26102 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 4.0.7.3-SNAPSHOT + 4.0.9-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -650,6 +650,11 @@ flexible messaging model and an intuitive client API. commons-collections commons-collections + + + io.opentelemetry + opentelemetry-common + 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 8848369f30277..54b8853fb0d71 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -172,7 +172,8 @@ public void asyncCreateLedger(int ensSize, int qSize, DigestType digestType, byt } @Override - public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx) { + public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx, + boolean keepUpdateMetadata) { getProgrammedFailure().thenComposeAsync((res) -> { PulsarMockLedgerHandle lh = ledgers.get(lId); if (lh == null) { From 6b76436f9e3c3d4db587f9124cb6a3b01f5b7026 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 23 Jan 2026 05:51:49 +0200 Subject: [PATCH 04/53] [fix][ml] Fix NoSuchElementException in EntryCountEstimator caused by a race condition (#25177) (cherry picked from commit 9b70ba303d5aa65c497bcdc4684cbc9e9017bb42) (cherry picked from commit 92618695322245a8d8e493a780a70735e66b4779) --- .../mledger/impl/EntryCountEstimator.java | 45 ++++++++++++++----- .../mledger/impl/EntryCountEstimatorTest.java | 40 +++++++++++++++++ 2 files changed, 74 insertions(+), 11 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java index 511379a7ae525..a5d41bad6a8d8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.Map; import java.util.NavigableMap; +import java.util.NoSuchElementException; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; @@ -82,17 +83,15 @@ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeByt return maxEntries; } - // Adjust the read position to ensure it falls within the valid range of available ledgers. - // This handles special cases such as EARLIEST and LATEST positions by resetting them - // to the first available ledger or the last active ledger, respectively. - if (lastLedgerId != null && readPosition.getLedgerId() > lastLedgerId.longValue()) { - readPosition = PositionFactory.create(lastLedgerId, Math.max(lastLedgerTotalEntries - 1, 0)); - } else if (lastLedgerId == null && readPosition.getLedgerId() > ledgersInfo.lastKey()) { - Map.Entry lastEntry = ledgersInfo.lastEntry(); - readPosition = - PositionFactory.create(lastEntry.getKey(), Math.max(lastEntry.getValue().getEntries() - 1, 0)); - } else if (readPosition.getLedgerId() < ledgersInfo.firstKey()) { - readPosition = PositionFactory.create(ledgersInfo.firstKey(), 0); + if (ledgersInfo.isEmpty()) { + return 1; + } + + try { + readPosition = adjustReadPosition(readPosition, ledgersInfo, lastLedgerId, lastLedgerTotalEntries); + } catch (NoSuchElementException e) { + // there was a race condition where ledgersInfo became empty just before adjustReadPosition was called + return 1; } long estimatedEntryCount = 0; @@ -183,4 +182,28 @@ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeByt // Ensure at least one entry is always returned as the result return Math.max((int) Math.min(estimatedEntryCount, maxEntries), 1); } + + private static Position adjustReadPosition(Position readPosition, + NavigableMap + ledgersInfo, + Long lastLedgerId, long lastLedgerTotalEntries) { + // Adjust the read position to ensure it falls within the valid range of available ledgers. + // This handles special cases such as EARLIEST and LATEST positions by resetting them + // to the first available ledger or the last active ledger, respectively. + if (lastLedgerId != null && readPosition.getLedgerId() > lastLedgerId.longValue()) { + return PositionFactory.create(lastLedgerId, Math.max(lastLedgerTotalEntries - 1, 0)); + } + long lastKey = ledgersInfo.lastKey(); + if (lastLedgerId == null && readPosition.getLedgerId() > lastKey) { + Map.Entry lastEntry = ledgersInfo.lastEntry(); + if (lastEntry != null && lastEntry.getKey() == lastKey) { + return PositionFactory.create(lastEntry.getKey(), Math.max(lastEntry.getValue().getEntries() - 1, 0)); + } + } + long firstKey = ledgersInfo.firstKey(); + if (readPosition.getLedgerId() < firstKey) { + return PositionFactory.create(firstKey, 0); + } + return readPosition; + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java index c1c1b8dd2c133..1b7bcbf816dc2 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java @@ -19,9 +19,12 @@ package org.apache.bookkeeper.mledger.impl; import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import java.util.HashSet; import java.util.NavigableMap; +import java.util.NoSuchElementException; import java.util.Set; import java.util.TreeMap; import org.apache.bookkeeper.mledger.Position; @@ -289,4 +292,41 @@ public void testMaxSizeIsLongMAX_VALUE() { int result = estimateEntryCountByBytesSize(Long.MAX_VALUE); assertEquals(result, maxEntries); } + + @Test + public void testNoLedgers() { + readPosition = PositionFactory.EARLIEST; + // remove all ledgers from ledgersInfo + ledgersInfo.clear(); + int result = estimateEntryCountByBytesSize(5_000_000); + // expect that result is 1 because the estimation couldn't be done + assertEquals(result, 1); + } + + @Test + public void testNoLedgersRaceFirstKey() { + readPosition = PositionFactory.EARLIEST; + // remove all ledgers from ledgersInfo + ledgersInfo = mock(NavigableMap.class); + when(ledgersInfo.isEmpty()).thenReturn(false); + when(ledgersInfo.firstKey()).thenThrow(NoSuchElementException.class); + when(ledgersInfo.lastKey()).thenReturn(1L); + int result = estimateEntryCountByBytesSize(5_000_000); + // expect that result is 1 because the estimation couldn't be done + assertEquals(result, 1); + } + + @Test + public void testNoLedgersRaceLastKey() { + readPosition = PositionFactory.EARLIEST; + // remove all ledgers from ledgersInfo + ledgersInfo = mock(NavigableMap.class); + lastLedgerId = null; + when(ledgersInfo.isEmpty()).thenReturn(false); + when(ledgersInfo.firstKey()).thenReturn(1L); + when(ledgersInfo.lastKey()).thenThrow(NoSuchElementException.class); + int result = estimateEntryCountByBytesSize(5_000_000); + // expect that result is 1 because the estimation couldn't be done + assertEquals(result, 1); + } } \ No newline at end of file From f574391beab5e9be673743be246cbc3e901f7401 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 27 Jan 2026 13:12:01 +0200 Subject: [PATCH 05/53] [fix][test] Bump org.assertj:assertj-core from 3.27.5 to 3.27.7 (#25186) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit ce4ebeaa7d1624cd4dd7814435700448e620e2d7) (cherry picked from commit 2c3402e56a5b086a05fb6c9feeb7870c6d8c0b6b) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 11166d5e26102..23548b2a0d0b4 100644 --- a/pom.xml +++ b/pom.xml @@ -276,7 +276,7 @@ flexible messaging model and an intuitive client API. 3.21.0 0.9.1 2.1.0 - 3.27.5 + 3.27.7 1.18.42 1.3.5 2.3.1 From c589d21225fe9934df3c60454c7c1a0eefcb90e1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 24 Jan 2026 10:11:10 +0200 Subject: [PATCH 06/53] [improve][misc] Upgrade snappy version to 1.1.10.8 (#25182) (cherry picked from commit b15f53b582546f47414edd9a680bd6980db2fd41) (cherry picked from commit 304fea1a88f5ca91f011d3dd622c554bf34c70f5) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index e2a36d69914a0..cd4105ffcf641 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -505,7 +505,7 @@ The Apache Software License, Version 2.0 - org.apache.zookeeper-zookeeper-jute-3.9.4.jar - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.4.jar * Snappy Java - - org.xerial.snappy-snappy-java-1.1.10.5.jar + - org.xerial.snappy-snappy-java-1.1.10.8.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 23548b2a0d0b4..b2bb780c4975b 100644 --- a/pom.xml +++ b/pom.xml @@ -175,14 +175,14 @@ flexible messaging model and an intuitive client API. package - 1.27.1 + 1.28.0 4.17.1.0.0.4 4.17.2 3.9.4 - 1.9.0 - 1.13.1 - 1.1.10.5 + 1.11.0 + 1.14.0 + 1.1.10.8 4.1.12.1 5.7.1 4.1.130.Final From 4eff75ab881f6c0e6aa449b4c9dfbacbd7badb0a Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 24 Jan 2026 11:20:40 +0800 Subject: [PATCH 07/53] [fix][proxy] Close client connection immediately when credentials expire and forwardAuthorizationCredentials is disabled (#25179) (cherry picked from commit 334847073e2c926eac5c486a5cfda1d1dd42634a) (cherry picked from commit c06f8baf70041ece152ea06d845e78b3bfdbdb1e) --- .../pulsar/proxy/server/ProxyConnection.java | 55 ++++-- .../proxy/server/ProxyAuthenticationTest.java | 175 ++++++++++++++++-- .../proxy/server/ProxyRefreshAuthTest.java | 48 ++--- 3 files changed, 216 insertions(+), 62 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 324a3ee9c87b4..d25a1d7286e74 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 @@ -148,7 +148,12 @@ enum State { Closing, - Closed, + Closed; + + boolean isAuthenticatedState() { + return this == ProxyLookupRequests + || this == ProxyConnectionToBroker; + } } ConnectionPool getConnectionPool() { @@ -417,15 +422,7 @@ private synchronized void completeConnect() throws PulsarClientException { "Failed to initialize lookup proxy handler")).addListener(ChannelFutureListener.CLOSE); return; } - if (service.getConfiguration().isAuthenticationEnabled() - && service.getConfiguration().getAuthenticationRefreshCheckSeconds() > 0) { - authRefreshTask = ctx.executor().scheduleAtFixedRate( - Runnables.catchingAndLoggingThrowables( - this::refreshAuthenticationCredentialsAndCloseIfTooExpired), - service.getConfiguration().getAuthenticationRefreshCheckSeconds(), - service.getConfiguration().getAuthenticationRefreshCheckSeconds(), - TimeUnit.SECONDS); - } + startAuthRefreshTaskIfNotStarted(); final ByteBuf msg = Commands.newConnected(protocolVersionToAdvertise, false); writeAndFlush(msg); } @@ -441,6 +438,10 @@ private void handleBrokerConnected(DirectProxyHandler directProxyHandler, Comman final ByteBuf msg = Commands.newConnected(connected.getProtocolVersion(), maxMessageSize, connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsTopicWatchers()); writeAndFlush(msg); + // Start auth refresh task only if we are not forwarding authorization credentials + if (!service.getConfiguration().isForwardAuthorizationCredentials()) { + startAuthRefreshTaskIfNotStarted(); + } } else { LOG.warn("[{}] Channel is {}. ProxyConnection is in {}. " + "Closing connection to broker '{}'.", @@ -522,16 +523,44 @@ protected void authChallengeSuccessCallback(AuthData authChallenge) { } } + private void startAuthRefreshTaskIfNotStarted() { + if (service.getConfiguration().isAuthenticationEnabled() + && service.getConfiguration().getAuthenticationRefreshCheckSeconds() > 0 + && authRefreshTask == null) { + authRefreshTask = ctx.executor().scheduleAtFixedRate( + Runnables.catchingAndLoggingThrowables( + this::refreshAuthenticationCredentialsAndCloseIfTooExpired), + service.getConfiguration().getAuthenticationRefreshCheckSeconds(), + service.getConfiguration().getAuthenticationRefreshCheckSeconds(), + TimeUnit.SECONDS); + } + } + private void refreshAuthenticationCredentialsAndCloseIfTooExpired() { assert ctx.executor().inEventLoop(); - if (state != State.ProxyLookupRequests) { - // Happens when an exception is thrown that causes this connection to close. + + // Only check expiration in authenticated states + if (!state.isAuthenticatedState()) { return; - } else if (!authState.isExpired()) { + } + + if (!authState.isExpired()) { // Credentials are still valid. Nothing to do at this point return; } + // If we are not forwarding authorization credentials to the broker, the broker cannot + // refresh the client's credentials. In this case, we must close the connection immediately + // when credentials expire. + if (!service.getConfiguration().isForwardAuthorizationCredentials()) { + if (LOG.isDebugEnabled()) { + LOG.debug("[{}] Closing connection because client credentials have expired and " + + "forwardAuthorizationCredentials is disabled (broker cannot refresh)", remoteAddress); + } + ctx.close(); + return; + } + if (System.nanoTime() - authChallengeSentTime > TimeUnit.SECONDS.toNanos(service.getConfiguration().getAuthenticationRefreshCheckSeconds())) { LOG.warn("[{}] Closing connection after timeout on refreshing auth credentials", remoteAddress); 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 6887e9ea234c1..04529629de7f1 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 @@ -18,11 +18,15 @@ */ package org.apache.pulsar.proxy.server; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; import com.google.gson.JsonObject; import com.google.gson.JsonParser; import java.io.IOException; +import java.net.SocketAddress; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -30,12 +34,16 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import javax.naming.AuthenticationException; +import javax.net.ssl.SSLSession; import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationDataCommand; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.authentication.AuthenticationState; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; @@ -44,8 +52,11 @@ 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.ProducerImpl; +import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.AuthAction; +import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -80,6 +91,7 @@ public boolean hasDataForHttp() { public Set> getHttpHeaders() { Map headers = new HashMap<>(); headers.put("BasicAuthentication", authParam); + headers.put("X-Pulsar-Auth-Method-Name", "BasicAuthentication"); return headers.entrySet(); } } @@ -119,6 +131,72 @@ public void start() throws PulsarClientException { } } + public static class BasicAuthenticationState implements AuthenticationState { + private final long expiryTimeInMillis; + private final String authRole; + private final AuthenticationDataSource authenticationDataSource; + + private static boolean isExpired(long expiryTimeInMillis) { + return System.currentTimeMillis() > expiryTimeInMillis; + } + + private static String[] parseAuthData(String commandData) { + JsonObject element = JsonParser.parseString(commandData).getAsJsonObject(); + long expiryTimeInMillis = Long.parseLong(element.get("expiryTime").getAsString()); + if (isExpired(expiryTimeInMillis)) { + throw new IllegalArgumentException("Credentials have expired"); + } + String role = element.get("entityType").getAsString(); + return new String[]{role, String.valueOf(expiryTimeInMillis)}; + } + + public BasicAuthenticationState(AuthenticationDataSource authData) { + this(authData.hasDataFromCommand() ? authData.getCommandData() + : authData.getHttpHeader("BasicAuthentication")); + } + + public BasicAuthenticationState(AuthData authData) { + this(new String(authData.getBytes(), StandardCharsets.UTF_8)); + } + + private BasicAuthenticationState(String commandData) { + String[] parsed = parseAuthData(commandData); + this.authRole = parsed[0]; + this.expiryTimeInMillis = Long.parseLong(parsed[1]); + this.authenticationDataSource = new AuthenticationDataCommand(commandData, null, null); + } + + @Override + public String getAuthRole() { + return authRole; + } + + @Override + public AuthData authenticate(AuthData authData) throws AuthenticationException { + return null; // Authentication complete + } + + @Override + public CompletableFuture authenticateAsync(AuthData authData) { + return CompletableFuture.completedFuture(null); // Authentication complete + } + + @Override + public AuthenticationDataSource getAuthDataSource() { + return authenticationDataSource; + } + + @Override + public boolean isComplete() { + return authRole != null; + } + + @Override + public boolean isExpired() { + return isExpired(expiryTimeInMillis); + } + } + public static class BasicAuthenticationProvider implements AuthenticationProvider { @Override @@ -135,26 +213,14 @@ public String getAuthMethodName() { } @Override - public CompletableFuture authenticateAsync(AuthenticationDataSource authData) { - String commandData = null; - if (authData.hasDataFromCommand()) { - commandData = authData.getCommandData(); - } else if (authData.hasDataFromHttp()) { - commandData = authData.getHttpHeader("BasicAuthentication"); - } + public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteAddress, SSLSession sslSession) { + return new BasicAuthenticationState(authData); + } - JsonObject element = JsonParser.parseString(commandData).getAsJsonObject(); - log.info("Have log of {}", element); - long expiryTimeInMillis = Long.parseLong(element.get("expiryTime").getAsString()); - long currentTimeInMillis = System.currentTimeMillis(); - if (expiryTimeInMillis < currentTimeInMillis) { - log.warn("Auth failed due to timeout"); - return CompletableFuture - .failedFuture(new AuthenticationException("Authentication data has been expired")); - } - final String result = element.get("entityType").getAsString(); - // Run in another thread to attempt to test the async logic - return CompletableFuture.supplyAsync(() -> result); + @Override + public CompletableFuture authenticateAsync(AuthenticationDataSource authData) { + BasicAuthenticationState basicAuthenticationState = new BasicAuthenticationState(authData); + return CompletableFuture.supplyAsync(basicAuthenticationState::getAuthRole); } } @@ -271,4 +337,75 @@ private PulsarClient createPulsarClient(String proxyServiceUrl, String authParam .authentication(BasicAuthentication.class.getName(), authParams) .connectionsPerBroker(numberOfConnections).build(); } + + @Test + void testClientDisconnectWhenCredentialsExpireWithoutForwardAuth() throws Exception { + log.info("-- Starting {} test --", methodName); + + String namespaceName = "my-property/my-ns"; + String topicName = "persistent://my-property/my-ns/my-topic1"; + + admin.namespaces().grantPermissionOnNamespace(namespaceName, "proxy", + Sets.newHashSet(AuthAction.consume, AuthAction.produce)); + admin.namespaces().grantPermissionOnNamespace(namespaceName, "client", + Sets.newHashSet(AuthAction.consume, AuthAction.produce)); + + // Important: When forwardAuthorizationCredentials=false, broker should not authenticate original auth data + // because the proxy doesn't forward it. Set authenticateOriginalAuthData=false to match this behavior. + conf.setAuthenticateOriginalAuthData(false); + + ProxyConfiguration proxyConfig = new ProxyConfiguration(); + proxyConfig.setAuthenticationEnabled(true); + proxyConfig.setAuthenticationRefreshCheckSeconds(2); // Check every 2 seconds + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setClusterName(CLUSTER_NAME); + + // Proxy auth with long expiry + String proxyAuthParams = "entityType:proxy,expiryTime:" + (System.currentTimeMillis() + 3600 * 1000); + proxyConfig.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); + proxyConfig.setBrokerClientAuthenticationParameters(proxyAuthParams); + + Set providers = new HashSet<>(); + providers.add(BasicAuthenticationProvider.class.getName()); + proxyConfig.setAuthenticationProviders(providers); + proxyConfig.setForwardAuthorizationCredentials(false); + + @Cleanup + AuthenticationService authenticationService = new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + 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(); + + // Create client with credentials that will expire in 3 seconds + long clientExpireTime = System.currentTimeMillis() + 3 * 1000; + String clientAuthParams = "entityType:client,expiryTime:" + clientExpireTime; + + @Cleanup + PulsarClient proxyClient = createPulsarClient(proxyServiceUrl, clientAuthParams, 1); + + @Cleanup + var producer = + proxyClient.newProducer(Schema.BYTES).topic(topicName).sendTimeout(5, TimeUnit.SECONDS).create(); + producer.send("test message".getBytes()); + + Awaitility.await().untilAsserted(() -> { + assertThatThrownBy(() -> producer.send("test message after expiry".getBytes())) + .isExactlyInstanceOf(PulsarClientException.TimeoutException.class); + }); + + if (producer instanceof ProducerImpl producerImpl) { + long lastDisconnectedTimestamp = producerImpl.getLastDisconnectedTimestamp(); + assertThat(lastDisconnectedTimestamp).isGreaterThan(clientExpireTime); + } + } } 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 f3ff1362fd895..7501eb9306f98 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 @@ -179,37 +179,25 @@ public void testAuthDataRefresh(boolean forwardAuthData) throws Exception { PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; pulsarClient.getPartitionsForTopic(topic).get(); - Set> connections = pulsarClientImpl.getCnxPool().getConnections(); - Awaitility.await().during(5, SECONDS).untilAsserted(() -> { - pulsarClient.getPartitionsForTopic(topic).get(); - assertTrue(connections.stream().allMatch(n -> { - try { - ClientCnx clientCnx = n.get(); - long timestamp = clientCnx.getLastDisconnectedTimestamp(); - return timestamp == 0; - } catch (Exception e) { - throw new RuntimeException(e); - } - })); - }); + // Verify initial connection state + Set> connections = pulsarClientImpl.getCnxPool().getConnections(); - // Force all connections from proxy to broker to close and therefore require the proxy to re-authenticate with - // the broker. (The client doesn't lose this connection.) - restartBroker(); - - // Rerun assertion to ensure that it still works - Awaitility.await().during(5, SECONDS).untilAsserted(() -> { - pulsarClient.getPartitionsForTopic(topic).get(); - assertTrue(connections.stream().allMatch(n -> { - try { - ClientCnx clientCnx = n.get(); - long timestamp = clientCnx.getLastDisconnectedTimestamp(); - return timestamp == 0; - } catch (Exception e) { - throw new RuntimeException(e); - } - })); - }); + Awaitility.await() + .during(5, SECONDS) + .untilAsserted(() -> { + for (CompletableFuture cf : connections) { + try { + ClientCnx clientCnx = cf.get(); + long timestamp = clientCnx.getLastDisconnectedTimestamp(); + // If forwardAuthData is false, the broker cannot see the client's authentication data. + // As a result, the broker cannot perform any refresh operations on the client's auth data. + // Only the proxy has visibility of the client's connection state. + assertTrue(forwardAuthData ? timestamp == 0 : timestamp > 0); + } catch (Exception e) { + throw new AssertionError("Failed to get connection state", e); + } + } + }); } } From 12268817e636f9617b6e21d36c50280047ee69ec Mon Sep 17 00:00:00 2001 From: Wenzhi Feng Date: Fri, 23 Jan 2026 17:42:32 +0800 Subject: [PATCH 08/53] [fix][client] ControlledClusterFailover avoid unnecessary reconnection. (#25178) Co-authored-by: fengwenzhi (cherry picked from commit f0ec07b3d8c5cfe36942957fc0ad32e40d69320d) (cherry picked from commit b41488d7e86d94703c40196894727d63ab02ddff) --- .../pulsar/client/impl/ControlledClusterFailover.java | 10 ++++++++++ .../client/impl/ControlledClusterFailoverTest.java | 11 +++++++++++ 2 files changed, 21 insertions(+) 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 c52a70c6cd70c..1d4740b847240 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 @@ -42,6 +42,7 @@ import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.asynchttpclient.AsyncHttpClient; @@ -109,6 +110,15 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, public void initialize(PulsarClient client) { this.pulsarClient = (PulsarClientImpl) client; + // Initialize currentControlledConfiguration from client's current configuration + // to avoid unnecessary reconnection on first scheduled check when the configuration hasn't changed + ClientConfigurationData conf = pulsarClient.getConfiguration(); + this.currentControlledConfiguration = new ControlledConfiguration(); + this.currentControlledConfiguration.setServiceUrl(currentPulsarServiceUrl); + this.currentControlledConfiguration.setTlsTrustCertsFilePath(conf.getTlsTrustCertsFilePath()); + this.currentControlledConfiguration.setAuthPluginClassName(conf.getAuthPluginClassName()); + this.currentControlledConfiguration.setAuthParamsString(conf.getAuthParams()); + // start to check service url every 30 seconds this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { ControlledConfiguration controlledConfiguration = 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 265af6dd23c07..86b2fa7cb4f9b 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 @@ -26,6 +26,7 @@ import lombok.Cleanup; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.asynchttpclient.Request; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -54,6 +55,14 @@ public void testBuildControlledClusterFailoverInstance() throws Exception { .build(); ControlledClusterFailover controlledClusterFailover = (ControlledClusterFailover) provider; + + PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + ClientConfigurationData clientConf = new ClientConfigurationData(); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); + when(pulsarClient.getConfiguration()).thenReturn(clientConf); + controlledClusterFailover.initialize(pulsarClient); + Request request = controlledClusterFailover.getRequestBuilder().build(); Assert.assertTrue(provider instanceof ControlledClusterFailover); @@ -91,7 +100,9 @@ public void testControlledClusterFailoverSwitch() throws Exception { ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); + ClientConfigurationData clientConf = new ClientConfigurationData(); when(pulsarClient.getCnxPool()).thenReturn(connectionPool); + when(pulsarClient.getConfiguration()).thenReturn(clientConf); controlledClusterFailover.initialize(pulsarClient); From 51787d9e63e551a66365d4f04c53752e32008e58 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 23 Jan 2026 08:47:37 +0200 Subject: [PATCH 09/53] [fix][sec] Bump org.apache.solr:solr-core from 9.8.0 to 9.10.1 in /pulsar-io/solr (#25175) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit a2f888a0d1e196f05c662f2771125d80d982c548) (cherry picked from commit b53206829c50f8c82dd1fc4692fe0ac2b4e0a6bd) --- pulsar-io/solr/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 8ac0da41e7971..91f4a13831498 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -29,7 +29,8 @@ - 9.8.0 + 9.10.1 + 10.0.24 pulsar-io-solr From 944d1ffd78cca37552cf88f2cc4c21b3c0795fce Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 26 Jan 2026 20:01:44 +0800 Subject: [PATCH 10/53] [improve][pip] PIP-453: Improve the metadata store threading model (#25173) (cherry picked from commit c51346fa3f5ec9cdd04ad03ba5d6b05b6c9a4f35) (cherry picked from commit d81d6b38becd86abbf4fbe3563ae852520f318a8) --- pip/pip-453.md | 98 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 98 insertions(+) create mode 100644 pip/pip-453.md diff --git a/pip/pip-453.md b/pip/pip-453.md new file mode 100644 index 0000000000000..a42736b9dda71 --- /dev/null +++ b/pip/pip-453.md @@ -0,0 +1,98 @@ +# PIP-453: Improve the metadata store threading model + +# Background knowledge + +The `pulsar-metadata` module provides two abstractions for interacting with metadata stores: +- `MetadataStore`: the wrapper on the actual underlying metadata store (e.g. ZooKeeper), which has caches for value and children of a given key. +- `MetadataCache`: a typed cache layer on top of `MetadataStore`, which performs serialization and deserialization of data between `T` and `byte[]`. + +The `MetadataStore` instance is unique in each broker, and is shared by multiple `MetadataCache` instances. + +However, a single thread whose name starts with the metadata store name (e.g. `ZK-MetadataStore`) is used by implementations of them. This thread is used in the following tasks: +1. Executing callbacks of APIs like `put`. +2. Executing notification handlers, including `AbstractMetadataStore#accept`, which calls `accept` methods of all `MetadataCache` instances and all listeners registered by `MetadataStore#registerListener`. +3. For ZooKeeper and Etcd, which support batching requests, it's used to schedule flushing tasks at a fixed rate, which is determined by the `metadataStoreBatchingMaxDelayMillis` config (default: 5 ms). +4. Scheduling some other tasks, e.g. retrying failed operations. + +It should be noted that `MetadataCache` executes the compute sensitive tasks like serialization in the `MetadataStore` callback. When the number of metadata operations grows, this thread is easy to be overwhelmed. It also affects the topic loading, which involves many metadata operations, this thread can be overwhelmed and block other tasks. For example, in a production environment, it's observed that the `pulsar_batch_metadata_store_queue_wait_time` metric is high (100 ms), which should be close to 5 ms normally (configured by `metadataStoreBatchingMaxDelayMillis`). + +# Motivation + +The single thread model is inefficient when there are many metadata operations. For example, when a broker is down and the topics owned by this broker will be transferred to the new owner broker. Since the new owner broker might never owned them before, even the `MetadataCache` caches are cold, which results in many metadata operations. However, the CPU-bound tasks like serialization and deserialization are executed in the `MetadataStore` thread, which makes it easy to be overwhelmed. This affects the topic loading time and the overall performance of the broker. + +In a production environment, there is a case when the metadata operation rate increased suddenly, the `pulsar_batch_metadata_store_queue_wait_time_ms_bucket` metric increased to ~100 ms, which is a part of the total latency of a metadata operation. As a result, the total P99 get latency (`pulsar_metadata_store_ops_latency_ms_bucket{type="get"}`) increased to 2 seconds. + +The 3rd task in the previous section is scheduled via `scheduleAtFixedRate`, which means if the task is not executed in time (5 ms by default), the task will be executed immediately again in a short time, which also burdens the single metadata store thread. + +# Goals + +## In Scope + +Improve the existing thread model to handle various tasks on metadata store, which could avoid a single thread being overwhelmed when there are many metadata operations. + +## Out of Scope + +Actually the batching mechanism introduced by [#13043](https://github.com/apache/pulsar/pull/13043) is harmful. The `flush` method, which is responsible to send a batch of metadata operations to broker, is called in the metadata store thread rather than the caller thread. The trade-off of the higher throughput is the lower latency. The benefit is limited because in most time the metadata operation rate is not so high. See this [test report](https://github.com/BewareMyPower/zookeeper-bench/blob/main/report.md) for more details. + +This proposal doesn't intend to change the existing batching mechanism or disable it by default. It only improves the threading model to avoid the single thread being overwhelmed. + +Additionally, some code paths execute the compute intensive tasks in the metadata store thread directly (e.g. `store.get(path).thenApply(/* ... */)`), this proposal does not aim at changing them to asynchronous methods (e.g. `thenApplyAsync`). + +# High Level Design + +Create 3 set of threads: +- `-event`: the original metadata store thread, which is now only responsible to handle notifications. This executor won't be a `ScheduledExecutorService` anymore. +- `-batch-flusher`: a single thread, which is used to schedule the flushing task at a fixed rate. It won't be created if `metadataStoreBatchingEnabled` is false. +- `-worker`: a fixed thread pool shared by all `MetadataCache` instances to execute compute intensive tasks like serialization and deserialization. The same path will be handled by the same thread to keep the processing order on the same path. + +Regarding the callbacks, don't switch to a different thread. This change is not breaking because the underlying metadata store usually executes the callback in a single thread (e.g. `-EventThread` in ZooKeeper) like the single thread in the current implementation. The caller should be responsible to manage worker threads on the metadata operation result if the callback is compute intensive. + +The only concern is that introducing a new thread to execute callbacks allows waiting for the future of metadata store APIs in the callback. After this change, the following use case could be a dead lock: + +```java +metadataStore.get(path).thenApply(__ -> metadataStore.get(otherPath).join());; +``` + +Other tasks like the retry on failure is executed in JVM's common `ForkJoinPool` by `CompletableFuture` APIs. For example: + +```diff +--- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java ++++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +@@ -245,9 +245,8 @@ public class ZKMetadataStore extends AbstractBatchedMetadataStore + countsByType, totalSize, opsForLog); + + // Retry with the individual operations +- executor.schedule(() -> { +- ops.forEach(o -> batchOperation(Collections.singletonList(o))); +- }, 100, TimeUnit.MILLISECONDS); ++ CompletableFuture.delayedExecutor(100, TimeUnit.MILLISECONDS).execute(() -> ++ ops.forEach(o -> batchOperation(Collections.singletonList(o)))); + } else { + MetadataStoreException e = getException(code, path); + ops.forEach(o -> o.getFuture().completeExceptionally(e)); +``` + +# Detailed Design + +## Public-facing Changes + +### Configuration + +Add a configurations to specify the number of worker threads for `MetadataCache`: + +```java + @FieldContext( + category = CATEGORY_SERVER, + doc = "The number of threads uses for serializing and deserializing data to and from the metadata store" + ) + private int metadataStoreSerDesThreads = Runtime.getRuntime().availableProcessors(); +``` + +### Metrics + +The `pulsar_batch_metadata_store_executor_queue_size` metric will be removed because the `-batch-flusher` thread won't execute other tasks except for flushing. + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/0cfdyvj96gw1sp1mo2zghl0lmsms5w1d +* Mailing List voting thread: https://lists.apache.org/thread/cktj2k8myw076yggn63k8yxs5357yd61 From ab607ecca288d485e84c050cced4ee8219a9cd1c Mon Sep 17 00:00:00 2001 From: zhenJiangWang Date: Tue, 27 Jan 2026 21:04:27 +0800 Subject: [PATCH 11/53] [improve][client]Reduce unnecessary getPartitionedTopicMetadata requests when using retry and DLQ topics. (#25172) (cherry picked from commit 52a4d5ee84fad6af2736376a6fcdd1bc41e7c52f) (cherry picked from commit 71a3994755063298368f738ea47882aba3f92f6a) --- .../client/api/DeadLetterTopicTest.java | 90 +++++++++++++++++++ .../client/impl/ConsumerBuilderImpl.java | 14 ++- 2 files changed, 102 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index f5bf31369d856..001edb4de4ff7 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 @@ -19,6 +19,15 @@ package org.apache.pulsar.client.api; 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.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.clearInvocations; +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.assertNotNull; import static org.testng.Assert.assertNull; @@ -48,7 +57,11 @@ import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.util.RetryMessageUtil; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.awaitility.Awaitility; import org.slf4j.Logger; @@ -1608,4 +1621,81 @@ public void sendDeadLetterTopicWithMismatchSchemaProducer() throws Exception { consumer.close(); deadLetterConsumer.close(); } + + @Test + public void testCheckUnnecessaryGetPartitionedTopicMetadataWhenUseRetryAndDQL() { + PulsarClientImpl client = mock(PulsarClientImpl.class); + ClientConfigurationData clientConf = new ClientConfigurationData(); + when(client.getConfiguration()).thenReturn(clientConf); + when(client.getPartitionedTopicMetadata(anyString(), anyBoolean(), anyBoolean())) + .thenReturn(CompletableFuture.completedFuture(new PartitionedTopicMetadata(0))); + when(client.subscribeAsync(any(ConsumerConfigurationData.class), any(), any())) + .thenReturn(CompletableFuture.completedFuture(mock(Consumer.class))); + + // Case 1: DeadLetterPolicy is null + ConsumerBuilderImpl consumerBuilder1 = new ConsumerBuilderImpl<>(client, Schema.BYTES); + consumerBuilder1.topic("persistent://public/default/test"); + consumerBuilder1.subscriptionName("sub"); + consumerBuilder1.enableRetry(true); + consumerBuilder1.subscribeAsync(); + + verify(client, times(1)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-RETRY"), anyBoolean(), anyBoolean()); + verify(client, times(1)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-DLQ"), anyBoolean(), anyBoolean()); + + clearInvocations(client); + + // Case 2: DeadLetterPolicy with custom Retry topic + ConsumerBuilderImpl consumerBuilder2 = new ConsumerBuilderImpl<>(client, Schema.BYTES); + consumerBuilder2.topic("persistent://public/default/test"); + consumerBuilder2.subscriptionName("sub"); + consumerBuilder2.enableRetry(true); + consumerBuilder2.deadLetterPolicy(DeadLetterPolicy.builder() + .maxRedeliverCount(10) + .retryLetterTopic("persistent://public/default/topic-retry") + .build()); + consumerBuilder2.subscribeAsync(); + + verify(client, times(0)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-RETRY"), anyBoolean(), anyBoolean()); + verify(client, times(1)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-DLQ"), anyBoolean(), anyBoolean()); + + clearInvocations(client); + + // Case 3: DeadLetterPolicy with custom DLQ topic + ConsumerBuilderImpl consumerBuilder3 = new ConsumerBuilderImpl<>(client, Schema.BYTES); + consumerBuilder3.topic("persistent://public/default/test"); + consumerBuilder3.subscriptionName("sub"); + consumerBuilder3.enableRetry(true); + consumerBuilder3.deadLetterPolicy(DeadLetterPolicy.builder() + .maxRedeliverCount(10) + .deadLetterTopic("persistent://public/default/topic-dlq") + .build()); + consumerBuilder3.subscribeAsync(); + + verify(client, times(1)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-RETRY"), anyBoolean(), anyBoolean()); + verify(client, times(0)).getPartitionedTopicMetadata( + eq("persistent://public/default/sub-DLQ"), anyBoolean(), anyBoolean()); + + clearInvocations(client); + + // Case 4: DeadLetterPolicy with both custom topics + ConsumerBuilderImpl consumerBuilder4 = new ConsumerBuilderImpl<>(client, Schema.BYTES); + consumerBuilder4.topic("persistent://public/default/test"); + consumerBuilder4.subscriptionName("sub"); + consumerBuilder4.enableRetry(true); + consumerBuilder4.deadLetterPolicy(DeadLetterPolicy.builder() + .maxRedeliverCount(10) + .retryLetterTopic("custom-retry") + .deadLetterTopic("custom-dlq") + .build()); + consumerBuilder4.subscribeAsync(); + + verify(client, times(0)).getPartitionedTopicMetadata(anyString(), anyBoolean(), anyBoolean()); + } + + } 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 ab64119cb8f29..50addea57d10d 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 @@ -163,8 +163,18 @@ public CompletableFuture> subscribeAsync() { DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy(); if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic()) || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { - CompletableFuture retryLetterTopicMetadata = checkDlqAlreadyExists(oldRetryLetterTopic); - CompletableFuture deadLetterTopicMetadata = checkDlqAlreadyExists(oldDeadLetterTopic); + CompletableFuture retryLetterTopicMetadata; + if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic())) { + retryLetterTopicMetadata = checkDlqAlreadyExists(oldRetryLetterTopic); + } else { + retryLetterTopicMetadata = CompletableFuture.completedFuture(false); + } + CompletableFuture deadLetterTopicMetadata; + if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { + deadLetterTopicMetadata = checkDlqAlreadyExists(oldDeadLetterTopic); + } else { + deadLetterTopicMetadata = CompletableFuture.completedFuture(false); + } applyDLQConfig = CompletableFuture.allOf(retryLetterTopicMetadata, deadLetterTopicMetadata) .thenAccept(__ -> { String retryLetterTopic = RetryMessageUtil.getRetryTopic(topicFirst.toString(), From 82a40864297de62f96345989c6b884e02b23a1de Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 30 Jan 2026 08:40:14 +0200 Subject: [PATCH 12/53] [fix][misc] Allow JWT tokens in OpenID auth without nbf claim (#25197) (cherry picked from commit d630394cdd02792b2dbc3a55443637a5d593a137) (cherry picked from commit 2760ee96caa7a02d7155d864b1b16ea773f520be) --- .../oidc/AuthenticationProviderOpenID.java | 1 - .../oidc/AuthenticationProviderOpenIDTest.java | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) 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 7f6f70c061571..cb7877543bf04 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 @@ -445,7 +445,6 @@ DecodedJWT verifyJWT(PublicKey publicKey, .withAnyOfAudience(allowedAudiences) .withClaimPresence(RegisteredClaims.ISSUED_AT) .withClaimPresence(RegisteredClaims.EXPIRES_AT) - .withClaimPresence(RegisteredClaims.NOT_BEFORE) .withClaimPresence(RegisteredClaims.SUBJECT); if (isRoleClaimNotSubject) { diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java index 377588c4a5ad8..27b3908eaa757 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.authentication.oidc; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertNull; import com.auth0.jwt.JWT; @@ -196,6 +197,20 @@ public void ensureFutureNBFFails() throws Exception { () -> basicProvider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.RS256.getValue(), jwt)); } + @Test + public void ensureWithoutNBFSucceeds() throws Exception { + KeyPair keyPair = Keys.keyPairFor(SignatureAlgorithm.RS256); + DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); + addValidMandatoryClaims(defaultJwtBuilder, basicProviderAudience); + // remove "nbf" claim + defaultJwtBuilder.setNotBefore(null); + defaultJwtBuilder.signWith(keyPair.getPrivate()); + DecodedJWT jwt = JWT.decode(defaultJwtBuilder.compact()); + assertThat(jwt.getNotBefore()).isNull(); + assertThat(jwt.getClaims().get("nbf")).isNull(); + basicProvider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.RS256.getValue(), jwt); + } + @Test public void ensureFutureIATFails() throws Exception { KeyPair keyPair = Keys.keyPairFor(SignatureAlgorithm.RS256); From b0ceda70274c89752f1a2855a30c814d4b23e945 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Thu, 5 Feb 2026 20:22:24 +0530 Subject: [PATCH 13/53] reverted-mismatched-conflicts-from-25166-commit --- pom.xml | 2 +- .../org/apache/bookkeeper/client/PulsarMockBookKeeper.java | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index b2bb780c4975b..c5bd795a12cba 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 4.0.9-SNAPSHOT + 4.0.7.3-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very 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 54b8853fb0d71..8848369f30277 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -172,8 +172,7 @@ public void asyncCreateLedger(int ensSize, int qSize, DigestType digestType, byt } @Override - public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx, - boolean keepUpdateMetadata) { + public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx) { getProgrammedFailure().thenComposeAsync((res) -> { PulsarMockLedgerHandle lh = ledgers.get(lId); if (lh == null) { From 9d88d1b024ce941ac212881c8e16d7b93940d4cf Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Thu, 5 Feb 2026 21:28:44 +0530 Subject: [PATCH 14/53] fix licence --- distribution/server/src/assemble/LICENSE.bin.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cd4105ffcf641..e281f3a3f266b 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -283,15 +283,15 @@ The Apache Software License, Version 2.0 - com.yahoo.datasketches-sketches-core-0.8.3.jar * Apache Commons - commons-beanutils-commons-beanutils-1.11.0.jar - - commons-cli-commons-cli-1.9.0.jar + - commons-cli-commons-cli-1.11.0.jar - commons-codec-commons-codec-1.18.0.jar - commons-io-commons-io-2.19.0.jar - commons-logging-commons-logging-1.3.5.jar - org.apache.commons-commons-collections4-4.5.0.jar - - org.apache.commons-commons-compress-1.27.1.jar + - org.apache.commons-commons-compress-1.28.0.jar - org.apache.commons-commons-configuration2-2.12.0.jar - org.apache.commons-commons-lang3-3.18.0.jar - - org.apache.commons-commons-text-1.13.1.jar + - org.apache.commons-commons-text-1.14.0.jar * Netty - io.netty-netty-buffer-4.1.130.Final.jar - io.netty-netty-codec-4.1.130.Final.jar @@ -540,7 +540,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar From f6116b6e679c465bd0996f0dc79ad6e768070bd7 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Thu, 5 Feb 2026 21:55:42 +0530 Subject: [PATCH 15/53] fix licence for commons codec io lang3 --- distribution/server/src/assemble/LICENSE.bin.txt | 6 +++--- pom.xml | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index e281f3a3f266b..8b8a313183359 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -284,13 +284,13 @@ The Apache Software License, Version 2.0 * Apache Commons - commons-beanutils-commons-beanutils-1.11.0.jar - commons-cli-commons-cli-1.11.0.jar - - commons-codec-commons-codec-1.18.0.jar - - commons-io-commons-io-2.19.0.jar + - commons-codec-commons-codec-1.20.0.jar + - commons-io-commons-io-2.21.0.jar - commons-logging-commons-logging-1.3.5.jar - org.apache.commons-commons-collections4-4.5.0.jar - org.apache.commons-commons-compress-1.28.0.jar - org.apache.commons-commons-configuration2-2.12.0.jar - - org.apache.commons-commons-lang3-3.18.0.jar + - org.apache.commons-commons-lang3-3.19.0.jar - org.apache.commons-commons-text-1.14.0.jar * Netty - io.netty-netty-buffer-4.1.130.Final.jar diff --git a/pom.xml b/pom.xml index c5bd795a12cba..6545790a94053 100644 --- a/pom.xml +++ b/pom.xml @@ -263,9 +263,9 @@ flexible messaging model and an intuitive client API. 7.9.2 0.27 2.12.4 - 3.18.0 - 2.19.0 - 1.18.0 + 3.19.0 + 2.21.0 + 1.20.0 1.3.5 2.1.6 2.1.9 From 21a64f91c8283564f1247a5045b2cf1fd381f975 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Mon, 9 Feb 2026 14:41:37 +0530 Subject: [PATCH 16/53] fix unit test update for commit 25132 --- .../loadbalance/extensions/ExtensibleLoadManagerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1244a3f1d1ad7..8b60e9c506659 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 @@ -1974,7 +1974,7 @@ SplitDecision.Reason.Unknown, new AtomicLong(6)) dimensions=[{broker=localhost, metric=sunitStateChn, result=Cancel}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=7}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Schedule}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=5}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Success}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=1}] - dimensions=[{broker=localhost, metric=sunitStateChn}], metrics=[{brk_sunit_state_chn_orphan_su_cleanup_ops_total=3, brk_sunit_state_chn_owned_su_total=10, brk_sunit_state_chn_su_tombstone_cleanup_ops_total=2}] + dimensions=[{broker=localhost, metric=sunitStateChn}], metrics=[{brk_sunit_state_chn_last_metadata_session_event_age_seconds=-1, brk_sunit_state_chn_last_metadata_session_event_is_reestablished=1, brk_sunit_state_chn_last_metadata_session_event_timestamp_ms=0, brk_sunit_state_chn_metadata_state=0, brk_sunit_state_chn_orphan_su_cleanup_ops_total=3, brk_sunit_state_chn_owned_su_total=10, brk_sunit_state_chn_su_tombstone_cleanup_ops_total=2}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_ignored_ack_total=3, brk_lb_ignored_send_total=2}] """.split("\n")); var actual = primaryLoadManager.getMetrics().stream().map(Metrics::toString).collect(Collectors.toSet()); From fcfb18bf36d142a8d1f54be1a203196823231e14 Mon Sep 17 00:00:00 2001 From: guptas6est Date: Mon, 2 Feb 2026 21:59:58 +0530 Subject: [PATCH 17/53] [fix][sec] Exclude org.lz4:lz4-java and standardize on at.yawk.lz4-java to remediate CVE-2025-12183 and CVE-2025-66566 (#25198) (cherry picked from commit c07f2adbaccc34c05bfbb895889f464fa9cc495f) (cherry picked from commit 2ac6d033e1ca2650e73296c9b47620b2ba932ce3) --- distribution/server/src/assemble/LICENSE.bin.txt | 3 ++- pom.xml | 7 +++++++ pulsar-common/pom.xml | 1 - pulsar-io/kafka-connect-adaptor/pom.xml | 4 ++++ pulsar-io/kinesis-kpl-shaded/pom.xml | 6 ++++++ pulsar-io/kinesis/pom.xml | 2 +- 6 files changed, 20 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8b8a313183359..750153cacb42a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -385,6 +385,7 @@ The Apache Software License, Version 2.0 - com.datastax.oss-bookkeeper-slogger-api-4.17.1.0.0.4.jar - com.datastax.oss-bookkeeper-slogger-slf4j-4.17.1.0.0.4.jar - com.datastax.oss-native-io-4.17.1.0.0.4.jar + - at.yawk.lz4-lz4-java-1.10.3.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -540,7 +541,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar diff --git a/pom.xml b/pom.xml index 6545790a94053..07d67e2b9ce93 100644 --- a/pom.xml +++ b/pom.xml @@ -374,6 +374,7 @@ flexible messaging model and an intuitive client API. 1.11.0 2.12.0 2.1.10 + 1.10.3 @@ -1766,6 +1767,12 @@ flexible messaging model and an intuitive client API. commons-configuration2 ${commons-configuration2.version} + + + at.yawk.lz4 + lz4-java + ${lz4java.version} + diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 61900b037ab1a..313c6a644b17d 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -225,7 +225,6 @@ at.yawk.lz4 lz4-java - 1.10.1 test diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 19dbdfc0203c9..ae19b31aba5a9 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -89,6 +89,10 @@ jose4j org.bitbucket.b_c + + org.lz4 + lz4-java + diff --git a/pulsar-io/kinesis-kpl-shaded/pom.xml b/pulsar-io/kinesis-kpl-shaded/pom.xml index 8ad2366439cb2..784ee2b06ea78 100644 --- a/pulsar-io/kinesis-kpl-shaded/pom.xml +++ b/pulsar-io/kinesis-kpl-shaded/pom.xml @@ -58,6 +58,12 @@ org.apache.kafka kafka-clients ${kafka-client.version} + + + org.lz4 + lz4-java + + diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index 861c1908085af..945caf3e8a1da 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -39,7 +39,7 @@ - + org.apache.kafka kafka-clients From 83f66b1dddd59d193b48fd54f5743a2e693212c4 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 4 Feb 2026 00:34:19 -0800 Subject: [PATCH 18/53] [fix] [test] Upgrade docker-java to 3.7.0 (#25209) (cherry picked from commit 4add84c709a0c88d2efcc922454a355c9ca15c5d) (cherry picked from commit 92b5d55468efe165c724cffe0631f90e97d3f313) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 07d67e2b9ce93..5ad5328bdf814 100644 --- a/pom.xml +++ b/pom.xml @@ -318,9 +318,9 @@ flexible messaging model and an intuitive client API. 3.3.2 - 1.21.3 + 1.21.4 - 3.4.2 + 3.7.0 2.2 5.4.0 1.1.1 From e89916c927933a7e9fedd19de68027ea032c077a Mon Sep 17 00:00:00 2001 From: Oneby Wang <44369297+oneby-wang@users.noreply.github.com> Date: Wed, 4 Feb 2026 18:46:37 +0800 Subject: [PATCH 19/53] [fix][client] Fix race condition between isDuplicate() and flushAsync() method in PersistentAcknowledgmentsGroupingTracker due to incorrect use Netty Recycler (#25208) (cherry picked from commit 5aab2f00f37c56303cb4efb86b3728d055eed7b1) (cherry picked from commit 22069491a9cff3947a538fb6b9c4169fa9a85f7b) --- .../pulsar/client/impl/ConsumerImpl.java | 11 ++-- ...sistentAcknowledgmentsGroupingTracker.java | 25 +++++----- .../AcknowledgementsGroupingTrackerTest.java | 50 +++++++++++++++++++ .../pulsar/common/protocol/Commands.java | 11 ++-- .../ConcurrentBitSetRecyclable.java | 1 + 5 files changed, 74 insertions(+), 24 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 81a421c9b3cc9..c1e8df4d35c19 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 @@ -136,7 +136,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.ConcurrentBitSet; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -3158,7 +3158,7 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me } else { if (Commands.peerSupportsMultiMessageAcknowledgment( getClientCnx().getRemoteEndpointProtocolVersion())) { - List> entriesToAck = + List> entriesToAck = new ArrayList<>(chunkMsgIds.length); for (MessageIdImpl cMsgId : chunkMsgIds) { if (cMsgId != null && chunkMsgIds.length > 1) { @@ -3196,7 +3196,7 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me } private ByteBuf newMultiTransactionMessageAck(long consumerId, TxnID txnID, - List> entries, + List> entries, long requestID) { BaseCommand cmd = newMultiMessageAckCommon(entries); cmd.getAck() @@ -3215,7 +3215,7 @@ protected BaseCommand initialValue() throws Exception { } }; - private static BaseCommand newMultiMessageAckCommon(List> entries) { + private static BaseCommand newMultiMessageAckCommon(List> entries) { BaseCommand cmd = LOCAL_BASE_COMMAND.get() .clear() .setType(BaseCommand.Type.ACK); @@ -3224,7 +3224,7 @@ private static BaseCommand newMultiMessageAckCommon(List pendingIndividualAcks; @VisibleForTesting - final ConcurrentSkipListMap pendingIndividualBatchIndexAcks; + final ConcurrentSkipListMap pendingIndividualBatchIndexAcks; private final ScheduledFuture scheduledTask; private final boolean batchIndexAckEnabled; @@ -133,7 +133,7 @@ public boolean isDuplicate(MessageId messageId) { return true; } if (messageIdAdv.getBatchIndex() >= 0) { - ConcurrentBitSetRecyclable bitSet = pendingIndividualBatchIndexAcks.get(key); + ConcurrentBitSet bitSet = pendingIndividualBatchIndexAcks.get(key); return bitSet != null && !bitSet.get(messageIdAdv.getBatchIndex()); } return false; @@ -327,21 +327,22 @@ private CompletableFuture doCumulativeAck(MessageIdAdv messageId, Map doIndividualBatchAckAsync(MessageIdAdv msgId) { - ConcurrentBitSetRecyclable bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( + ConcurrentBitSet bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( MessageIdAdvUtils.discardBatch(msgId), __ -> { final BitSet ackSet = msgId.getAckSet(); - final ConcurrentBitSetRecyclable value; + final ConcurrentBitSet value; if (ackSet != null) { synchronized (ackSet) { if (!ackSet.isEmpty()) { - value = ConcurrentBitSetRecyclable.create(ackSet); + value = new ConcurrentBitSet(); + value.or(ackSet); } else { - value = ConcurrentBitSetRecyclable.create(); + value = new ConcurrentBitSet(); value.set(0, msgId.getBatchSize()); } } } else { - value = ConcurrentBitSetRecyclable.create(); + value = new ConcurrentBitSet(); value.set(0, msgId.getBatchSize()); } return value; @@ -445,7 +446,7 @@ private void flushAsync(ClientCnx cnx) { } // Flush all individual acks - List> entriesToAck = + List> entriesToAck = new ArrayList<>(pendingIndividualAcks.size() + pendingIndividualBatchIndexAcks.size()); if (!pendingIndividualAcks.isEmpty()) { if (Commands.peerSupportsMultiMessageAcknowledgment(cnx.getRemoteEndpointProtocolVersion())) { @@ -487,7 +488,7 @@ private void flushAsync(ClientCnx cnx) { } while (true) { - Map.Entry entry = + Map.Entry entry = pendingIndividualBatchIndexAcks.pollFirstEntry(); if (entry == null) { // The entry has been removed in a different thread @@ -539,7 +540,7 @@ private CompletableFuture newImmediateAckAndFlush(long consumerId, Message // cumulative ack chunk by the last messageId if (chunkMsgIds != null && ackType != AckType.Cumulative) { if (Commands.peerSupportsMultiMessageAcknowledgment(cnx.getRemoteEndpointProtocolVersion())) { - List> entriesToAck = new ArrayList<>(chunkMsgIds.length); + 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)); @@ -568,7 +569,7 @@ private CompletableFuture newMessageAckCommandAndWrite( long entryId, BitSetRecyclable ackSet, AckType ackType, Map properties, boolean flush, TimedCompletableFuture timedCompletableFuture, - List> entriesToAck) { + List> entriesToAck) { if (consumer.isAckReceiptEnabled()) { final long requestId = consumer.getClient().newRequestId(); final ByteBuf cmd; 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 8c7a605cfea5e..76f2f45a0304c 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 @@ -31,8 +31,10 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; +import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; @@ -406,6 +408,54 @@ public void testDoIndividualBatchAckAsync() { tracker.close(); } + @Test + public void testDoIndividualBatchAckNeverAffectIsDuplicate() throws Exception { + ConsumerConfigurationData conf = new ConsumerConfigurationData<>(); + conf.setMaxAcknowledgmentGroupSize(1); + PersistentAcknowledgmentsGroupingTracker tracker = + new PersistentAcknowledgmentsGroupingTracker(consumer, conf, eventLoopGroup); + + BatchMessageIdImpl batchMessageId0 = new BatchMessageIdImpl(5, 1, 0, 0, 10, null); + BatchMessageIdImpl batchMessageId1 = new BatchMessageIdImpl(5, 1, 0, 1, 10, null); + + int loops = 10000; + int addAcknowledgmentThreadCount = 10; + List addAcknowledgmentThreads = new ArrayList<>(addAcknowledgmentThreadCount); + for (int i = 0; i < addAcknowledgmentThreadCount; i++) { + Thread addAcknowledgmentThread = new Thread(() -> { + for (int j = 0; j < loops; j++) { + tracker.addAcknowledgment(batchMessageId0, AckType.Individual, Collections.emptyMap()); + } + }, "doIndividualBatchAck-thread-" + i); + addAcknowledgmentThread.start(); + addAcknowledgmentThreads.add(addAcknowledgmentThread); + } + + int isDuplicateThreadCount = 10; + AtomicBoolean assertResult = new AtomicBoolean(); + List isDuplicateThreads = new ArrayList<>(isDuplicateThreadCount); + for (int i = 0; i < isDuplicateThreadCount; i++) { + Thread isDuplicateThread = new Thread(() -> { + for (int j = 0; j < loops; j++) { + boolean duplicate = tracker.isDuplicate(batchMessageId1); + assertResult.set(assertResult.get() || duplicate); + } + }, "isDuplicate-thread-" + i); + isDuplicateThread.start(); + isDuplicateThreads.add(isDuplicateThread); + } + + for (Thread addAcknowledgmentThread : addAcknowledgmentThreads) { + addAcknowledgmentThread.join(); + } + + for (Thread isDuplicateThread : isDuplicateThreads) { + isDuplicateThread.join(); + } + + assertFalse(assertResult.get()); + } + public class ClientCnxTest extends ClientCnx { public ClientCnxTest(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { 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 07ad634440d53..bdbaacc8a9412 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 @@ -109,7 +109,7 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; +import org.apache.pulsar.common.util.collections.ConcurrentBitSet; @UtilityClass @Slf4j @@ -1009,7 +1009,7 @@ public static ByteBuf newLookupErrorResponse(ServerError error, String errorMsg, } public static ByteBuf newMultiTransactionMessageAck(long consumerId, TxnID txnID, - List> entries) { + List> entries) { BaseCommand cmd = newMultiMessageAckCommon(entries); cmd.getAck() .setConsumerId(consumerId) @@ -1019,14 +1019,14 @@ public static ByteBuf newMultiTransactionMessageAck(long consumerId, TxnID txnID return serializeWithSize(cmd); } - private static BaseCommand newMultiMessageAckCommon(List> entries) { + private static BaseCommand newMultiMessageAckCommon(List> entries) { BaseCommand cmd = localCmd(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(); + ConcurrentBitSet bitSet = entries.get(i).getRight(); MessageIdData msgId = ack.addMessageId() .setLedgerId(ledgerId) .setEntryId(entryId); @@ -1035,7 +1035,6 @@ private static BaseCommand newMultiMessageAckCommon(List> entries, + List> entries, long requestId) { BaseCommand cmd = newMultiMessageAckCommon(entries); cmd.getAck() diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java index 0ba409b2d7d17..d29e4b8240fde 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java @@ -27,6 +27,7 @@ /** * Safe multithreaded version of {@code BitSet} and leverage netty recycler. */ +@Deprecated @EqualsAndHashCode(callSuper = true) public class ConcurrentBitSetRecyclable extends ConcurrentBitSet { From b036f398473ad3e98837128e57cde435222756a1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 4 Feb 2026 15:07:07 +0200 Subject: [PATCH 20/53] [fix][build][branch-4.0] Fix build after invalid cherry-pick merge conflict resolution in 2ac6d03 (cherry picked from commit e2d65d6adb2259497b0eb2081d268e828ea1c15a) From e29b2ed91e4e9c4121d7d3c517de9343546710db Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Mon, 9 Feb 2026 21:48:36 +0530 Subject: [PATCH 21/53] fix licence and pom for lz4-java --- distribution/server/src/assemble/LICENSE.bin.txt | 3 +-- pom.xml | 6 ++---- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 750153cacb42a..8b8a313183359 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -385,7 +385,6 @@ The Apache Software License, Version 2.0 - com.datastax.oss-bookkeeper-slogger-api-4.17.1.0.0.4.jar - com.datastax.oss-bookkeeper-slogger-slf4j-4.17.1.0.0.4.jar - com.datastax.oss-native-io-4.17.1.0.0.4.jar - - at.yawk.lz4-lz4-java-1.10.3.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -541,7 +540,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar diff --git a/pom.xml b/pom.xml index 5ad5328bdf814..3f69efe61342d 100644 --- a/pom.xml +++ b/pom.xml @@ -227,7 +227,7 @@ flexible messaging model and an intuitive client API. 3.11.2 4.5.0 3.9.1 - 1.10.1 + 1.10.3 5.28.0 1.12.788 2.32.28 @@ -374,7 +374,6 @@ flexible messaging model and an intuitive client API. 1.11.0 2.12.0 2.1.10 - 1.10.3 @@ -1767,11 +1766,10 @@ flexible messaging model and an intuitive client API. commons-configuration2 ${commons-configuration2.version} - at.yawk.lz4 lz4-java - ${lz4java.version} + ${lz4-java.version} From 2c4ede0cfb8a0ac9d018ba5d534b465ec351e42b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 18 Nov 2025 14:14:55 +0800 Subject: [PATCH 22/53] [improve][monitor] Upgrade OpenTelemetry to 1.56.0, Otel instrumentation to 2.21.0 and Otel semconv to 1.37.0 (#24994) (cherry picked from commit 53162ff745f3381ad97afeebd613c291e7b4bd2e) (cherry picked from commit a1d5b6ce3b744704b233fc0ebbd02e2429d29804) --- .../server/src/assemble/LICENSE.bin.txt | 66 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 7 +- pom.xml | 30 ++++----- .../opentelemetry/OpenTelemetryService.java | 18 ++--- .../OpenTelemetryServiceTest.java | 13 ++-- 5 files changed, 63 insertions(+), 71 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8b8a313183359..78a02b769a3ff 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -338,12 +338,12 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar * Prometheus exporter - - io.prometheus-prometheus-metrics-config-1.3.4.jar - - io.prometheus-prometheus-metrics-exporter-common-1.3.4.jar - - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.4.jar - - io.prometheus-prometheus-metrics-exposition-formats-1.3.4.jar - - io.prometheus-prometheus-metrics-model-1.3.4.jar - - io.prometheus-prometheus-metrics-exposition-textformats-1.3.4.jar + - io.prometheus-prometheus-metrics-config-1.3.10.jar + - io.prometheus-prometheus-metrics-exporter-common-1.3.10.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.10.jar + - io.prometheus-prometheus-metrics-exposition-formats-no-protobuf-1.3.10.jar + - io.prometheus-prometheus-metrics-model-1.3.10.jar + - io.prometheus-prometheus-metrics-exposition-textformats-1.3.10.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -418,17 +418,16 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - com.google.errorprone-error_prone_annotations-2.45.0.jar * Apache Thrift - org.apache.thrift-libthrift-0.14.2.jar * OkHttp3 - - com.squareup.okhttp3-logging-interceptor-4.9.3.jar - - com.squareup.okhttp3-okhttp-4.9.3.jar + - com.squareup.okhttp3-logging-interceptor-5.3.1.jar + - com.squareup.okhttp3-okhttp-5.3.1.jar + - com.squareup.okhttp3-okhttp-jvm-5.3.1.jar * Okio - - com.squareup.okio-okio-3.4.0.jar - - com.squareup.okio-okio-jvm-3.4.0.jar + - com.squareup.okio-okio-3.16.3.jar + - com.squareup.okio-okio-jvm-3.16.3.jar * Javassist -- org.javassist-javassist-3.25.0-GA.jar * Kotlin Standard Lib - org.jetbrains.kotlin-kotlin-stdlib-1.8.20.jar - org.jetbrains.kotlin-kotlin-stdlib-common-1.8.20.jar - - org.jetbrains.kotlin-kotlin-stdlib-jdk7-1.8.20.jar - - org.jetbrains.kotlin-kotlin-stdlib-jdk8-1.8.20.jar - org.jetbrains-annotations-13.0.jar * gRPC - io.grpc-grpc-all-1.75.0.jar @@ -519,27 +518,28 @@ The Apache Software License, Version 2.0 * RoaringBitmap - org.roaringbitmap-RoaringBitmap-1.2.0.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.45.0.jar - - io.opentelemetry-opentelemetry-api-incubator-1.45.0-alpha.jar - - io.opentelemetry-opentelemetry-context-1.45.0.jar - - io.opentelemetry-opentelemetry-exporter-common-1.45.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.45.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.45.0.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.45.0-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-common-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.45.0.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.45.0.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.6.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.6-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.6-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.6-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.6-alpha.jar - - io.opentelemetry.semconv-opentelemetry-semconv-1.29.0-alpha.jar + - io.opentelemetry-opentelemetry-api-1.56.0.jar + - io.opentelemetry-opentelemetry-api-incubator-1.56.0-alpha.jar + - io.opentelemetry-opentelemetry-common-1.56.0.jar + - io.opentelemetry-opentelemetry-context-1.56.0.jar + - io.opentelemetry-opentelemetry-exporter-common-1.56.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.56.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.56.0.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.56.0-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-common-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.56.0.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.56.0.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-2.21.0.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-incubator-2.21.0-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-resources-2.21.0-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-2.21.0-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-2.21.0-alpha.jar + - io.opentelemetry.semconv-opentelemetry-semconv-1.37.0.jar - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 7af948c69be53..2e4dd89ca159e 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -385,9 +385,10 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.25.3.jar - log4j-web-2.25.3.jar * OpenTelemetry - - opentelemetry-api-1.45.0.jar - - opentelemetry-api-incubator-1.45.0-alpha.jar - - opentelemetry-context-1.45.0.jar + - opentelemetry-api-1.56.0.jar + - opentelemetry-api-incubator-1.56.0-alpha.jar + - opentelemetry-common-1.56.0.jar + - opentelemetry-context-1.56.0.jar * BookKeeper - bookkeeper-common-allocator-4.17.1.0.0.4.jar diff --git a/pom.xml b/pom.xml index 3f69efe61342d..19fdf999368a1 100644 --- a/pom.xml +++ b/pom.xml @@ -285,10 +285,10 @@ flexible messaging model and an intuitive client API. 2.0.2 5.12.1 18.0.0 - 0.9.6 - 4.9.3 + 0.9.4 + 5.3.1 - 3.4.0 + 3.16.3 1.8.20 1.0 @@ -307,11 +307,11 @@ flexible messaging model and an intuitive client API. the core logic is switched to java implementation of zstd in org.apache.commons:commons-compress --> 1.5.7-3 2.0.6 - 1.45.0 + 1.56.0 ${opentelemetry.version}-alpha - 1.33.6 + 2.21.0 ${opentelemetry.instrumentation.version}-alpha - 1.29.0-alpha + 1.37.0 4.7.5 1.8 0.3.6 @@ -1635,23 +1635,17 @@ flexible messaging model and an intuitive client API. com.squareup.okhttp3 - okhttp - ${okhttp3.version} - - - com.squareup.okhttp3 - okhttp-urlconnection - ${okhttp3.version} - - - com.squareup.okhttp3 - logging-interceptor + okhttp-bom ${okhttp3.version} + pom + import com.squareup.okio - okio + okio-bom ${okio.version} + pom + import diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index e6c6d95273e0e..d143b743d3497 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -28,7 +28,7 @@ import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.common.export.MemoryMode; import io.opentelemetry.sdk.resources.Resource; -import io.opentelemetry.semconv.ResourceAttributes; +import io.opentelemetry.semconv.ServiceAttributes; import java.io.Closeable; import java.util.Map; import java.util.Objects; @@ -74,9 +74,9 @@ public OpenTelemetryService(String clusterName, sdkBuilder.addPropertiesSupplier(() -> Map.of( OTEL_SDK_DISABLED_KEY, "true", // Cardinality limit includes the overflow attribute set, so we need to add 1. - "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1), + "otel.java.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1), // Reduce number of allocations by using reusable data mode. - "otel.java.experimental.exporter.memory_mode", MemoryMode.REUSABLE_DATA.name() + "otel.java.exporter.memory_mode", MemoryMode.REUSABLE_DATA.name() )); sdkBuilder.addResourceCustomizer( @@ -87,13 +87,13 @@ public OpenTelemetryService(String clusterName, resourceBuilder.put(OpenTelemetryAttributes.PULSAR_CLUSTER, clusterName); } if (StringUtils.isNotBlank(serviceName) - && Objects.equals(Resource.getDefault().getAttribute(ResourceAttributes.SERVICE_NAME), - resource.getAttribute(ResourceAttributes.SERVICE_NAME))) { - resourceBuilder.put(ResourceAttributes.SERVICE_NAME, serviceName); + && Objects.equals(Resource.getDefault().getAttribute(ServiceAttributes.SERVICE_NAME), + resource.getAttribute(ServiceAttributes.SERVICE_NAME))) { + resourceBuilder.put(ServiceAttributes.SERVICE_NAME, serviceName); } if (StringUtils.isNotBlank(serviceVersion) - && resource.getAttribute(ResourceAttributes.SERVICE_VERSION) == null) { - resourceBuilder.put(ResourceAttributes.SERVICE_VERSION, serviceVersion); + && resource.getAttribute(ServiceAttributes.SERVICE_VERSION) == null) { + resourceBuilder.put(ServiceAttributes.SERVICE_VERSION, serviceVersion); } return resource.merge(resourceBuilder.build()); }); @@ -123,7 +123,7 @@ public OpenTelemetryService(String clusterName, // disable JFR based telemetry and use only JMX telemetry .disableAllFeatures() // enable experimental JMX telemetry in addition - .enableExperimentalJmxTelemetry() + .emitExperimentalTelemetry() .build()); } diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index 99d4189d8f803..e3dd29cd1b23e 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.opentelemetry; import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; -import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.satisfies; import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.LongCounterBuilder; @@ -29,7 +28,7 @@ import io.opentelemetry.sdk.common.InstrumentationScopeInfo; import io.opentelemetry.sdk.metrics.export.MetricReader; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; -import io.opentelemetry.semconv.ResourceAttributes; +import io.opentelemetry.semconv.ServiceAttributes; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -38,7 +37,6 @@ import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; -import org.assertj.core.api.AbstractCharSequenceAssert; import org.awaitility.Awaitility; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -98,7 +96,7 @@ public void testClusterNameCannotBeEmpty() { } @Test - public void testResourceAttributesAreSet() throws Exception { + public void testServiceAttributesAreSet() throws Exception { @Cleanup var reader = InMemoryMetricReader.create(); @@ -116,9 +114,8 @@ public void testResourceAttributesAreSet() throws Exception { .allSatisfy(metric -> assertThat(metric) .hasResourceSatisfying(resource -> resource .hasAttribute(OpenTelemetryAttributes.PULSAR_CLUSTER, "testServiceNameAndVersion") - .hasAttribute(ResourceAttributes.SERVICE_NAME, "openTelemetryServiceTestService") - .hasAttribute(ResourceAttributes.SERVICE_VERSION, "1.0.0") - .hasAttribute(satisfies(ResourceAttributes.HOST_NAME, AbstractCharSequenceAssert::isNotBlank)))); + .hasAttribute(ServiceAttributes.SERVICE_NAME, "openTelemetryServiceTestService") + .hasAttribute(ServiceAttributes.SERVICE_VERSION, "1.0.0"))); } @Test @@ -230,7 +227,7 @@ public void testJvmRuntimeMetrics() { // Buffer Pool Metrics // Replaces jvm_buffer_pool_used_bytes - assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.memory.usage")); + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.memory.used")); // Replaces jvm_buffer_pool_capacity_bytes assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.memory.limit")); // Replaces jvm_buffer_pool_used_buffers From b403caed4997d17d46a07c7fa137ce6362322252 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 5 Feb 2026 17:34:07 +0800 Subject: [PATCH 23/53] [improve][proxy] Add regression tests for package upload with 'Expect: 100-continue' (#25211) Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> (cherry picked from commit e8fedb16ca6b8b02b4981b325899000f1c828395) (cherry picked from commit 09476393e8fc3df065b7a3a31b8ee126c620fa24) --- .../core/MockedPackagesStorage.java | 10 +- .../core/MockedPackagesStorageTest.java | 52 +++++++ pulsar-proxy/pom.xml | 8 + .../proxy/server/AdminProxyHandler.java | 79 +--------- .../proxy/server/ProxyPackagesUploadTest.java | 144 ++++++++++++++++++ 5 files changed, 220 insertions(+), 73 deletions(-) create mode 100644 pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorageTest.java create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPackagesUploadTest.java diff --git a/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorage.java b/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorage.java index 1a4b8010d51bd..6e76d142c1266 100644 --- a/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorage.java +++ b/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorage.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.packages.management.core; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -45,8 +46,13 @@ public CompletableFuture writeAsync(String path, InputStream inputStream) CompletableFuture future = new CompletableFuture<>(); CompletableFuture.runAsync(() -> { try { - byte[] bytes = new byte[inputStream.available()]; - inputStream.read(bytes); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + byte[] buffer = new byte[8192]; + int read; + while ((read = inputStream.read(buffer)) != -1) { + baos.write(buffer, 0, read); + } + byte[] bytes = baos.toByteArray(); storage.put(path, bytes); future.complete(null); } catch (IOException e) { diff --git a/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorageTest.java b/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorageTest.java new file mode 100644 index 0000000000000..eb48f02680db6 --- /dev/null +++ b/pulsar-package-management/core/src/test/java/org/apache/pulsar/packages/management/core/MockedPackagesStorageTest.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.packages.management.core; + +import static org.mockito.Mockito.mock; +import static org.testng.Assert.assertEquals; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import lombok.Cleanup; +import org.testng.annotations.Test; + +public class MockedPackagesStorageTest { + + @Test + public void testWriteAndRead() throws Exception { + PackagesStorageProvider provider = new MockedPackagesStorageProvider(); + PackagesStorage storage = provider.getStorage(mock(PackagesStorageConfiguration.class)); + storage.initialize(); + + // Test data + byte[] testBytes = new byte[1 * 1024 * 1024]; + + // Write + storage.writeAsync("test/path", new ByteArrayInputStream(testBytes)).get(); + + // Read + @Cleanup + ByteArrayOutputStream readBaos = new ByteArrayOutputStream(); + storage.readAsync("test/path", readBaos).get(); + + // Verify + assertEquals(readBaos.toByteArray(), testBytes); + + storage.closeAsync().get(); + } +} diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 0682151120a3f..a1ac3893c4cb3 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -227,6 +227,14 @@ test-jar test + + + ${project.groupId} + pulsar-package-core + ${project.version} + test-jar + test + 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 8d0c2c3ae397a..08f87f3d08a29 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,12 +29,10 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Set; -import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import javax.net.ssl.SSLContext; -import javax.servlet.ServletConfig; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -56,9 +54,7 @@ import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP; import org.eclipse.jetty.http.HttpHeader; import org.eclipse.jetty.proxy.ProxyServlet; -import org.eclipse.jetty.util.HttpCookieStore; import org.eclipse.jetty.util.ssl.SslContextFactory; -import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,75 +114,16 @@ class AdminProxyHandler extends ProxyServlet { @Override protected HttpClient createHttpClient() throws ServletException { - ServletConfig config = getServletConfig(); - - HttpClient client = newHttpClient(); - - client.setFollowRedirects(true); - - // Must not store cookies, otherwise cookies of different clients will mix. - client.setCookieStore(new HttpCookieStore.Empty()); - - Executor executor; - String value = config.getInitParameter("maxThreads"); - if (value == null || "-".equals(value)) { - executor = (Executor) getServletContext().getAttribute("org.eclipse.jetty.server.Executor"); - if (executor == null) { - throw new IllegalStateException("No server executor for proxy"); - } - } else { - QueuedThreadPool qtp = new QueuedThreadPool(Integer.parseInt(value)); - String servletName = config.getServletName(); - int dot = servletName.lastIndexOf('.'); - if (dot >= 0) { - servletName = servletName.substring(dot + 1); - } - qtp.setName(servletName); - executor = qtp; - } - - client.setExecutor(executor); - - value = config.getInitParameter("maxConnections"); - if (value == null) { - value = "256"; - } - client.setMaxConnectionsPerDestination(Integer.parseInt(value)); - - value = config.getInitParameter("idleTimeout"); - if (value == null) { - value = "30000"; - } - client.setIdleTimeout(Long.parseLong(value)); - - value = config.getInitParameter(INIT_PARAM_REQUEST_BUFFER_SIZE); - if (value != null) { - client.setRequestBufferSize(Integer.parseInt(value)); - } - - value = config.getInitParameter("responseBufferSize"); - if (value != null){ - client.setResponseBufferSize(Integer.parseInt(value)); - } - - try { - 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. - ProtocolHandlers protocolHandlers = client.getProtocolHandlers(); - protocolHandlers.clear(); - protocolHandlers.put(new RedirectProtocolHandler(client)); - - return client; - } catch (Exception x) { - throw new ServletException(x); - } + HttpClient httpClient = super.createHttpClient(); + customizeHttpClient(httpClient); + return httpClient; } + private void customizeHttpClient(HttpClient httpClient) { + httpClient.setFollowRedirects(true); + ProtocolHandlers protocolHandlers = httpClient.getProtocolHandlers(); + protocolHandlers.put(new RedirectProtocolHandler(httpClient)); + } // This class allows the request body to be replayed, the default implementation // does not diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPackagesUploadTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPackagesUploadTest.java new file mode 100644 index 0000000000000..8575be212070a --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPackagesUploadTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import static com.google.common.net.HttpHeaders.EXPECT; +import static org.assertj.core.api.Assertions.assertThat; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Optional; +import lombok.Cleanup; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.packages.management.core.MockedPackagesStorageProvider; +import org.apache.pulsar.packages.management.core.common.PackageMetadata; +import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; +import org.asynchttpclient.RequestBuilder; +import org.asynchttpclient.Response; +import org.asynchttpclient.request.body.multipart.FilePart; +import org.asynchttpclient.request.body.multipart.StringPart; +import org.eclipse.jetty.servlet.ServletHolder; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +public class ProxyPackagesUploadTest extends MockedPulsarServiceBaseTest { + + private static final int FILE_SIZE = 8 * 1024 * 1024; // 8 MB + private static final ObjectMapper MAPPER = ObjectMapperFactory.create(); + private WebServer webServer; + private PulsarAdmin proxyAdmin; + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + conf.setEnablePackagesManagement(true); + conf.setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName()); + super.internalSetup(); + + ProxyConfiguration proxyConfig = new ProxyConfiguration(); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setBrokerWebServiceURL(brokerUrl.toString()); + + webServer = new WebServer(proxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig, true))); + webServer.addServlet("/", new ServletHolder(new AdminProxyHandler(proxyConfig, null, null))); + webServer.start(); + + proxyAdmin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + webServer.getListenPortHTTP().get()) + .build(); + + admin.tenants().createTenant("public", createDefaultTenantInfo()); + admin.namespaces().createNamespace("public/default"); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + if (proxyAdmin != null) { + proxyAdmin.close(); + } + if (webServer != null) { + webServer.stop(); + } + super.internalCleanup(); + } + + @Test + public void testUploadPackageThroughProxy() throws Exception { + Path packageFile = Files.createTempFile("pkg-sdk", ".nar"); + packageFile.toFile().deleteOnExit(); + Files.write(packageFile, new byte[FILE_SIZE]); + + String pkgName = "function://public/default/pkg-sdk@v1"; + PackageMetadata meta = PackageMetadata.builder().description("sdk-test").build(); + + proxyAdmin.packages().upload(meta, pkgName, packageFile.toString()); + + verifyDownload(pkgName, FILE_SIZE); + } + + @Test + public void testUploadWithExpect100Continue() throws Exception { + Path packageFile = Files.createTempFile("pkg-ahc", ".nar"); + packageFile.toFile().deleteOnExit(); + Files.write(packageFile, new byte[FILE_SIZE]); + + String pkgName = "function://public/default/expect-test@v1"; + String uploadUrl = String.format("http://localhost:%d/admin/v3/packages/function/public/default/expect-test/v1", + webServer.getListenPortHTTP().orElseThrow()); + + @Cleanup + AsyncHttpClient client = new DefaultAsyncHttpClient(new DefaultAsyncHttpClientConfig.Builder().build()); + + Response response = client.executeRequest(new RequestBuilder("POST") + .setUrl(uploadUrl) + .addHeader(EXPECT, "100-continue") + .addBodyPart(new FilePart("file", packageFile.toFile())) + .addBodyPart(new StringPart("metadata", MAPPER.writeValueAsString( + PackageMetadata.builder().description("ahc-test").build()), "application/json")) + .build()).get(); + + assertThat(response.getStatusCode()).isEqualTo(204); + + verifyDownload(pkgName, FILE_SIZE); + } + + private void verifyDownload(String packageName, int expectedSize) throws Exception { + Path fromBroker = Files.createTempFile("from-broker", ".nar"); + fromBroker.toFile().deleteOnExit(); + admin.packages().download(packageName, fromBroker.toString()); + assertThat(Files.size(fromBroker)).isEqualTo(expectedSize); + Files.deleteIfExists(fromBroker); + + Path fromProxy = Files.createTempFile("from-proxy", ".nar"); + fromProxy.toFile().deleteOnExit(); + proxyAdmin.packages().download(packageName, fromProxy.toString()); + assertThat(Files.size(fromProxy)).isEqualTo(expectedSize); + } +} From e55d90276270d4221e1bcd480f3b3349384b794c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 4 Feb 2026 20:01:35 +0800 Subject: [PATCH 24/53] [improve][meta] PIP-453: Improve the metadata store threading model (#25187) (cherry picked from commit 6d415c6dbfb8bf9f2f8a5b823927cf422a6bd675) (cherry picked from commit c5f7271499e8879353139b9b1ee021a8dda111ad) --- conf/broker.conf | 2 + conf/standalone.conf | 3 + pip/pip-453.md | 26 +-- ...IsolatedBookieEnsemblePlacementPolicy.java | 3 + .../pulsar/broker/ServiceConfiguration.java | 6 + ...atedBookieEnsemblePlacementPolicyTest.java | 34 ++-- .../apache/pulsar/broker/PulsarService.java | 2 + .../pulsar/broker/PulsarServiceTest.java | 65 ++++++ .../OpenTelemetryMetadataStoreStatsTest.java | 12 -- .../metadata/api/MetadataStoreConfig.java | 3 + .../cache/impl/MetadataCacheImpl.java | 185 +++++++++++------- .../metadata/impl/AbstractMetadataStore.java | 77 +++++--- .../metadata/impl/EtcdMetadataStore.java | 19 +- .../impl/LocalMemoryMetadataStore.java | 2 +- .../metadata/impl/RocksdbMetadataStore.java | 2 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 19 +- .../AbstractBatchedMetadataStore.java | 39 ++-- .../metadata/impl/oxia/OxiaMetadataStore.java | 4 +- .../impl/stats/BatchMetadataStoreStats.java | 43 +--- .../impl/MetadataStoreFactoryImplTest.java | 2 +- 20 files changed, 316 insertions(+), 232 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 7ab535d18e90e..d751d423ba42d 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -920,6 +920,8 @@ metadataStoreBatchingMaxOperations=1000 # Maximum size of a batch metadataStoreBatchingMaxSizeKb=128 +# The number of threads used for serializing and deserializing data to and from the metadata store +metadataStoreSerDesThreads=1 ### --- Authentication --- ### diff --git a/conf/standalone.conf b/conf/standalone.conf index fd5f9f11d3174..abd4c60836958 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -430,6 +430,9 @@ metadataStoreBatchingMaxOperations=1000 # Maximum size of a batch metadataStoreBatchingMaxSizeKb=128 +# The number of threads used for serializing and deserializing data to and from the metadata store +metadataStoreSerDesThreads=1 + ### --- TLS --- ### # Deprecated - Use webServicePortTls and brokerServicePortTls instead tlsEnabled=false diff --git a/pip/pip-453.md b/pip/pip-453.md index a42736b9dda71..f9109798ba778 100644 --- a/pip/pip-453.md +++ b/pip/pip-453.md @@ -40,8 +40,9 @@ Additionally, some code paths execute the compute intensive tasks in the metadat # High Level Design -Create 3 set of threads: +Create 4 sets of threads: - `-event`: the original metadata store thread, which is now only responsible to handle notifications. This executor won't be a `ScheduledExecutorService` anymore. +- `-scheduler`: a single thread, which is used to schedule tasks like flushing and retrying failed operations. - `-batch-flusher`: a single thread, which is used to schedule the flushing task at a fixed rate. It won't be created if `metadataStoreBatchingEnabled` is false. - `-worker`: a fixed thread pool shared by all `MetadataCache` instances to execute compute intensive tasks like serialization and deserialization. The same path will be handled by the same thread to keep the processing order on the same path. @@ -53,25 +54,6 @@ The only concern is that introducing a new thread to execute callbacks allows wa metadataStore.get(path).thenApply(__ -> metadataStore.get(otherPath).join());; ``` -Other tasks like the retry on failure is executed in JVM's common `ForkJoinPool` by `CompletableFuture` APIs. For example: - -```diff ---- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java -+++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java -@@ -245,9 +245,8 @@ public class ZKMetadataStore extends AbstractBatchedMetadataStore - countsByType, totalSize, opsForLog); - - // Retry with the individual operations -- executor.schedule(() -> { -- ops.forEach(o -> batchOperation(Collections.singletonList(o))); -- }, 100, TimeUnit.MILLISECONDS); -+ CompletableFuture.delayedExecutor(100, TimeUnit.MILLISECONDS).execute(() -> -+ ops.forEach(o -> batchOperation(Collections.singletonList(o)))); - } else { - MetadataStoreException e = getException(code, path); - ops.forEach(o -> o.getFuture().completeExceptionally(e)); -``` - # Detailed Design ## Public-facing Changes @@ -85,9 +67,11 @@ Add a configurations to specify the number of worker threads for `MetadataCache` category = CATEGORY_SERVER, doc = "The number of threads uses for serializing and deserializing data to and from the metadata store" ) - private int metadataStoreSerDesThreads = Runtime.getRuntime().availableProcessors(); + private int metadataStoreSerDesThreads = 1; ``` +Use 1 as the default value since the serialization and deserialization tasks are not frequent. This separated thread pool is mainly added to avoid blocking the metadata store callback thread. + ### Metrics The `pulsar_batch_metadata_store_executor_queue_size` metric will be removed because the `-batch-flusher` thread won't execute other tasks except for flushing. 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 878bbc4d654a7..4ef1c594be444 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 @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; @@ -57,6 +58,8 @@ public class IsolatedBookieEnsemblePlacementPolicy extends RackawareEnsemblePlac // the secondary group. private ImmutablePair, Set> defaultIsolationGroups; + @Getter + @VisibleForTesting private MetadataCache bookieMappingCache; private static final String PULSAR_SYSTEM_TOPIC_ISOLATION_GROUP = "*"; 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 bb9a38905e21e..4b4c0d787d49a 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 @@ -484,6 +484,12 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private boolean metadataStoreAllowReadOnlyOperations; + @FieldContext( + category = CATEGORY_SERVER, + doc = "The number of threads used for serializing and deserializing data to and from the metadata store" + ) + private int metadataStoreSerDesThreads = 1; + @Deprecated @FieldContext( category = CATEGORY_SERVER, 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 68f92ab416dc2..936b04386ff7b 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 @@ -22,12 +22,15 @@ 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.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.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -288,8 +291,7 @@ public void testBasic() throws Exception { secondaryBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("group2", secondaryBookieGroup); - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); ensemble = isolationPolicy.newEnsemble(2, 2, 2, Collections.emptyMap(), null).getResult(); @@ -340,8 +342,7 @@ public void testNoBookieInfo() throws Exception { + "\": {\"rack\": \"rack0\", \"hostname\": \"bookie3.example.com\"}, \"" + BOOKIE4 + "\": {\"rack\": \"rack2\", \"hostname\": \"bookie4.example.com\"}}}"; - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, data.getBytes(StandardCharsets.UTF_8), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, data.getBytes(StandardCharsets.UTF_8)); List ensemble = isolationPolicy.newEnsemble(2, 2, 2, Collections.emptyMap(), new HashSet<>()).getResult(); @@ -399,8 +400,7 @@ public void testBookieInfoChange() throws Exception { bookieMapping.put("group1", mainBookieGroup); bookieMapping.put("group2", secondaryBookieGroup); - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); ensemble = isolationPolicy.newEnsemble(3, 3, 3, Collections.emptyMap(), new HashSet<>()).getResult(); @@ -784,8 +784,7 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup2, group2); bookieMapping.put(isolationGroup3, group3); - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); groups.setLeft(Sets.newHashSet(isolationGroup1, isolationGroup3)); groups.setRight(Sets.newHashSet("")); @@ -808,8 +807,7 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup1, group1); bookieMapping.put(isolationGroup2, group2); - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); groups.setLeft(Sets.newHashSet(isolationGroup1)); groups.setRight(Sets.newHashSet(isolationGroup2)); @@ -831,12 +829,24 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup1, group1); bookieMapping.put(isolationGroup2, group2); - store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), - Optional.empty()).join(); + updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); groups.setLeft(Sets.newHashSet(isolationGroup1)); groups.setRight(Sets.newHashSet(isolationGroup2)); blacklist = isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); assertTrue(blacklist.isEmpty()); } + + // The policy gets the bookie info asynchronously before each query or update, when putting the bookie info into + // the metadata store, the cache needs some time to receive the notification and update accordingly. + private void updateBookieInfo(IsolatedBookieEnsemblePlacementPolicy isolationPolicy, byte[] bookieInfo) { + final var cache = isolationPolicy.getBookieMappingCache(); + assertNotNull(cache); // the policy must have been initialized + + final var key = BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH; + final var previousBookieInfo = cache.getIfCached(key); + store.put(key, bookieInfo, Optional.empty()).join(); + Awaitility.await().atMost(Duration.ofSeconds(1)).untilAsserted(() -> + assertNotEquals(cache.getIfCached(key), previousBookieInfo)); + } } 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 fd9c74b3e82bd..53907fe8fb495 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 @@ -429,6 +429,7 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro .synchronizer(synchronizer) .openTelemetry(openTelemetry) .nodeSizeStats(new DefaultMetadataNodeSizeStats()) + .numSerDesThreads(config.getMetadataStoreSerDesThreads()) .build()); } @@ -1299,6 +1300,7 @@ public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchro .metadataStoreName(MetadataStoreConfig.METADATA_STORE) .openTelemetry(openTelemetry) .nodeSizeStats(new DefaultMetadataNodeSizeStats()) + .numSerDesThreads(config.getMetadataStoreSerDesThreads()) .build()); } 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 6c04889d8f1ba..6195e9cdae593 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 @@ -25,10 +25,15 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertSame; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -38,6 +43,10 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; +import org.apache.pulsar.metadata.api.MetadataSerde; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.Stat; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; @@ -339,4 +348,60 @@ public void testShutdownViaAdminApi() throws Exception { assertTrue(e instanceof PulsarClientException.TimeoutException); } } + + @Test + public void testMetadataSerDesThreads() throws Exception { + final var numSerDesThreads = 5; + final var config = new ServiceConfiguration(); + config.setMetadataStoreSerDesThreads(numSerDesThreads); + config.setClusterName("test"); + config.setMetadataStoreUrl("memory:local"); + config.setConfigurationMetadataStoreUrl("memory:local"); + + @Cleanup final var pulsar = new PulsarService(config); + pulsar.start(); + + BiConsumer verifier = (store, prefix) -> { + final var serDes = new CustomMetadataSerDes(); + final var cache = store.getMetadataCache(prefix, serDes, MetadataCacheConfig.builder().build()); + for (int i = 0; i < 100 && serDes.threadNameToSerializedPaths.size() < numSerDesThreads; i++) { + cache.create(prefix + i, "value-" + i).join(); + final var value = cache.get(prefix + i).join(); + assertEquals(value.orElseThrow(), "value-" + i); + final var newValue = cache.readModifyUpdate(prefix + i, s -> s + "-updated").join(); + assertEquals(newValue, "value-" + i + "-updated"); + // Verify the serialization and deserialization are handled by the same thread + assertEquals(serDes.threadNameToSerializedPaths, serDes.threadNameToDeserializedPaths); + } + log.info("SerDes thread mapping: {}", serDes.threadNameToSerializedPaths); + assertEquals(serDes.threadNameToSerializedPaths.keySet().size(), numSerDesThreads); + // Verify a path cannot be handled by multiple threads + final var paths = serDes.threadNameToSerializedPaths.values().stream() + .flatMap(Set::stream).sorted().toList(); + assertEquals(paths.stream().distinct().toList(), paths); + }; + + verifier.accept(pulsar.getLocalMetadataStore(), "/test-local/"); + verifier.accept(pulsar.getConfigurationMetadataStore(), "/test-config/"); + } + + private static class CustomMetadataSerDes implements MetadataSerde { + + final Map> threadNameToSerializedPaths = new ConcurrentHashMap<>(); + final Map> threadNameToDeserializedPaths = new ConcurrentHashMap<>(); + + @Override + public byte[] serialize(String path, String value) throws IOException{ + threadNameToSerializedPaths.computeIfAbsent(Thread.currentThread().getName(), + __ -> ConcurrentHashMap.newKeySet()).add(path); + return value.getBytes(); + } + + @Override + public String deserialize(String path, byte[] data, Stat stat) throws IOException { + threadNameToDeserializedPaths.computeIfAbsent(Thread.currentThread().getName(), + __ -> ConcurrentHashMap.newKeySet()).add(path); + return new String(data); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java index 9e8bde20b88e7..390aa1e49e29d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java @@ -21,7 +21,6 @@ import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.assertj.core.api.Assertions.assertThat; import io.opentelemetry.api.common.Attributes; -import java.util.concurrent.ExecutorService; import lombok.Cleanup; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.BrokerTestUtil; @@ -29,7 +28,6 @@ import org.apache.pulsar.broker.testcontext.NonClosingProxyHandler; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.metadata.api.MetadataStore; -import org.apache.pulsar.metadata.impl.stats.BatchMetadataStoreStats; import org.apache.pulsar.metadata.impl.stats.MetadataStoreStats; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -53,14 +51,6 @@ protected void setup() throws Exception { var newStats = new MetadataStoreStats( localMetadataStoreName, pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); FieldUtils.writeField(localMetadataStore, "metadataStoreStats", newStats, true); - - var currentBatchedStats = (BatchMetadataStoreStats) FieldUtils.readField(localMetadataStore, - "batchMetadataStoreStats", true); - currentBatchedStats.close(); - var currentExecutor = (ExecutorService) FieldUtils.readField(currentBatchedStats, "executor", true); - var newBatchedStats = new BatchMetadataStoreStats(localMetadataStoreName, currentExecutor, - pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); - FieldUtils.writeField(localMetadataStore, "batchMetadataStoreStats", newBatchedStats, true); } @AfterMethod(alwaysRun = true) @@ -89,7 +79,5 @@ public void testMetadataStoreStats() throws Exception { var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); assertMetricLongSumValue(metrics, MetadataStoreStats.METADATA_STORE_PUT_BYTES_COUNTER_METRIC_NAME, attributes, value -> assertThat(value).isPositive()); - assertMetricLongSumValue(metrics, BatchMetadataStoreStats.EXECUTOR_QUEUE_SIZE_METRIC_NAME, attributes, - value -> assertThat(value).isPositive()); } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java index ef50dc87691d0..fcde0dce8404b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java @@ -104,4 +104,7 @@ public class MetadataStoreConfig { * The estimator to estimate the payload length of metadata node, which used to limit the batch size requested. */ private MetadataNodeSizeStats nodeSizeStats; + + @Builder.Default + private final int numSerDesThreads = 1; } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index b1f0572547ca7..ca165f0464e44 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -39,10 +39,10 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.pulsar.common.stats.CacheMetricsCollector; import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.metadata.api.CacheGetResult; -import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataSerde; @@ -62,23 +62,26 @@ public class MetadataCacheImpl implements MetadataCache, Consumer serde; - private final ScheduledExecutorService executor; + private final OrderedExecutor executor; + private final ScheduledExecutorService schedulerExecutor; private final MetadataCacheConfig cacheConfig; private final AsyncLoadingCache>> objCache; public MetadataCacheImpl(String cacheName, MetadataStore store, TypeReference typeRef, - MetadataCacheConfig cacheConfig, ScheduledExecutorService executor) { - this(cacheName, store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor); + MetadataCacheConfig cacheConfig, OrderedExecutor executor, + ScheduledExecutorService schedulerExecutor) { + this(cacheName, store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor, schedulerExecutor); } public MetadataCacheImpl(String cacheName, MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig, - ScheduledExecutorService executor) { - this(cacheName, store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor); + OrderedExecutor executor, ScheduledExecutorService schedulerExecutor) { + this(cacheName, store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor, schedulerExecutor); } public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde serde, - MetadataCacheConfig cacheConfig, ScheduledExecutorService executor) { + MetadataCacheConfig cacheConfig, OrderedExecutor executor, + ScheduledExecutorService schedulerExecutor) { this.store = store; if (store instanceof MetadataStoreExtended) { this.storeExtended = (MetadataStoreExtended) store; @@ -88,6 +91,7 @@ public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde this.serde = serde; this.cacheConfig = cacheConfig; this.executor = executor; + this.schedulerExecutor = schedulerExecutor; Caffeine cacheBuilder = Caffeine.newBuilder(); if (cacheConfig.getRefreshAfterWriteMillis() > 0) { @@ -101,6 +105,9 @@ public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde .buildAsync(new AsyncCacheLoader>>() { @Override public CompletableFuture>> asyncLoad(String key, Executor executor) { + if (log.isDebugEnabled()) { + log.debug("Loading key {} into metadata cache {}", key, cacheName); + } return readValueFromStore(key); } @@ -110,12 +117,16 @@ public CompletableFuture>> asyncReload( Optional> oldValue, Executor executor) { if (store instanceof AbstractMetadataStore && ((AbstractMetadataStore) store).isConnected()) { - return readValueFromStore(key).thenApply(val -> { + if (log.isDebugEnabled()) { + log.debug("Reloading key {} into metadata cache {}", key, cacheName); + } + final var future = readValueFromStore(key); + future.thenAccept(val -> { if (cacheConfig.getAsyncReloadConsumer() != null) { cacheConfig.getAsyncReloadConsumer().accept(key, val); } - return val; }); + return future; } else { // Do not try to refresh the cache item if we know that we're not connected to the // metadata store @@ -128,22 +139,46 @@ public CompletableFuture>> asyncReload( } private CompletableFuture>> readValueFromStore(String path) { - return store.get(path) - .thenCompose(optRes -> { - if (!optRes.isPresent()) { - return FutureUtils.value(Optional.empty()); - } - - try { - GetResult res = optRes.get(); - T obj = serde.deserialize(path, res.getValue(), res.getStat()); - return FutureUtils - .value(Optional.of(new CacheGetResult<>(obj, res.getStat()))); - } catch (Throwable t) { - return FutureUtils.exception(new ContentDeserializationException( - "Failed to deserialize payload for key '" + path + "'", t)); - } - }); + final var future = new CompletableFuture>>(); + store.get(path).thenComposeAsync(optRes -> { + // There could be multiple pending reads for the same path, for example, when a path is created, + // 1. The `accept` method will call `refresh` + // 2. The `put` method will call `refresh` after the metadata store put operation is done + // Both will call this method and the same result will be read. In this case, we only need to deserialize + // the value once. + if (!optRes.isPresent()) { + if (log.isDebugEnabled()) { + log.debug("Key {} not found in metadata store", path); + } + return FutureUtils.value(Optional.>empty()); + } + final var res = optRes.get(); + final var cachedFuture = objCache.getIfPresent(path); + if (cachedFuture != null && cachedFuture != future) { + if (log.isDebugEnabled()) { + log.debug("A new read on key {} is in progress or completed, ignore this one", path); + } + return cachedFuture; + } + try { + T obj = serde.deserialize(path, res.getValue(), res.getStat()); + if (log.isDebugEnabled()) { + log.debug("Deserialized value for key {} (version: {}): {}", path, res.getStat().getVersion(), + obj); + } + return FutureUtils.value(Optional.of(new CacheGetResult<>(obj, res.getStat()))); + } catch (Throwable t) { + return FutureUtils.exception(new ContentDeserializationException( + "Failed to deserialize payload for key '" + path + "'", t)); + } + }, executor.chooseThread(path)).whenComplete((result, e) -> { + if (e != null) { + future.completeExceptionally(e.getCause()); + } else { + future.complete(result); + } + }); + return future; } @Override @@ -169,8 +204,9 @@ public Optional getIfCached(String path) { @Override public CompletableFuture readModifyUpdateOrCreate(String path, Function, T> modifyFunction) { + final var executor = this.executor.chooseThread(path); return executeWithRetry(() -> objCache.get(path) - .thenCompose(optEntry -> { + .thenComposeAsync(optEntry -> { Optional currentValue; long expectedVersion; @@ -202,13 +238,14 @@ public CompletableFuture readModifyUpdateOrCreate(String path, Function { refresh(path); }).thenApply(__ -> newValueObj); - }), path); + }, executor), path); } @Override public CompletableFuture readModifyUpdate(String path, Function modifyFunction) { + final var executor = this.executor.chooseThread(path); return executeWithRetry(() -> objCache.get(path) - .thenCompose(optEntry -> { + .thenComposeAsync(optEntry -> { if (!optEntry.isPresent()) { return FutureUtils.exception(new NotFoundException("")); } @@ -231,59 +268,57 @@ public CompletableFuture readModifyUpdate(String path, Function modifyF return store.put(path, newValue, Optional.of(expectedVersion)).thenAccept(__ -> { refresh(path); }).thenApply(__ -> newValueObj); - }), path); + }, executor), path); + } + + private CompletableFuture serialize(String path, T value) { + final var future = new CompletableFuture(); + executor.executeOrdered(path, () -> { + try { + future.complete(serde.serialize(path, value)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + }); + return future; } @Override public CompletableFuture create(String path, T value) { - byte[] content; - try { - content = serde.serialize(path, value); - } catch (Throwable t) { - return FutureUtils.exception(t); - } - - CompletableFuture future = new CompletableFuture<>(); - store.put(path, content, Optional.of(-1L)) - .thenAccept(stat -> { - // Make sure we have the value cached before the operation is completed - // In addition to caching the value, we need to add a watch on the path, - // so when/if it changes on any other node, we are notified and we can - // update the cache - objCache.get(path).whenComplete((stat2, ex) -> { - if (ex == null) { - future.complete(null); - } else { - log.error("Exception while getting path {}", path, ex); - future.completeExceptionally(ex.getCause()); - } - }); - }).exceptionally(ex -> { - if (ex.getCause() instanceof BadVersionException) { - // Use already exists exception to provide more self-explanatory error message - future.completeExceptionally(new AlreadyExistsException(ex.getCause())); - } else { - future.completeExceptionally(ex.getCause()); - } - return null; - }); - + final var future = new CompletableFuture(); + serialize(path, value).thenCompose(content -> store.put(path, content, Optional.of(-1L))) + // Make sure we have the value cached before the operation is completed + // In addition to caching the value, we need to add a watch on the path, + // so when/if it changes on any other node, we are notified and we can + // update the cache + .thenCompose(__ -> objCache.get(path)) + .whenComplete((__, ex) -> { + if (ex == null) { + future.complete(null); + } else if (ex.getCause() instanceof BadVersionException) { + // Use already exists exception to provide more self-explanatory error message + future.completeExceptionally(new AlreadyExistsException(ex.getCause())); + } else { + future.completeExceptionally(ex.getCause()); + } + }); return future; } @Override public CompletableFuture put(String path, T value, EnumSet options) { - final byte[] bytes; - try { - bytes = serde.serialize(path, value); - } catch (IOException e) { - return CompletableFuture.failedFuture(e); - } - if (storeExtended != null) { - return storeExtended.put(path, bytes, Optional.empty(), options).thenAccept(__ -> refresh(path)); - } else { - return store.put(path, bytes, Optional.empty()).thenAccept(__ -> refresh(path)); - } + return serialize(path, value).thenCompose(bytes -> { + if (storeExtended != null) { + return storeExtended.put(path, bytes, Optional.empty(), options); + } else { + return store.put(path, bytes, Optional.empty()); + } + }).thenAccept(__ -> { + if (log.isDebugEnabled()) { + log.debug("Refreshing path {} after put operation", path); + } + refresh(path); + }); } @Override @@ -323,6 +358,9 @@ public void accept(Notification t) { switch (t.getType()) { case Created: case Modified: + if (log.isDebugEnabled()) { + log.debug("Refreshing path {} for {} notification", path, t.getType()); + } refresh(path); break; @@ -354,8 +392,7 @@ private void execute(Supplier> op, String key, CompletableF final var next = backoff.next(); log.info("Update key {} conflicts. Retrying in {} ms. Mandatory stop: {}. Elapsed time: {} ms", key, next, backoff.isMandatoryStopMade(), elapsed); - executor.schedule(() -> execute(op, key, result, backoff), next, - TimeUnit.MILLISECONDS); + schedulerExecutor.schedule(() -> execute(op, key, result, backoff), next, TimeUnit.MILLISECONDS); return null; } result.completeExceptionally(ex.getCause()); 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 b0e4b43f70067..d118a792e2f23 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 @@ -26,6 +26,7 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.MoreExecutors; import io.netty.util.concurrent.DefaultThreadFactory; import io.opentelemetry.api.OpenTelemetry; import java.time.Instant; @@ -38,16 +39,17 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; -import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.stats.CacheMetricsCollector; import org.apache.pulsar.common.util.FutureUtil; @@ -76,7 +78,9 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList> sessionListeners = new CopyOnWriteArrayList<>(); protected final String metadataStoreName; - protected final ScheduledExecutorService executor; + private final OrderedExecutor serDesExecutor; + private final ExecutorService eventExecutor; + private final ScheduledExecutorService schedulerExecutor; private final AsyncLoadingCache> childrenCache; private final AsyncLoadingCache existsCache; private final CopyOnWriteArrayList> metadataCaches = new CopyOnWriteArrayList<>(); @@ -93,13 +97,21 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co protected MetadataNodeSizeStats nodeSizeStats; - protected AbstractMetadataStore(String metadataStoreName, OpenTelemetry openTelemetry, - MetadataNodeSizeStats nodeSizeStats) { + protected AbstractMetadataStore( + String metadataStoreName, OpenTelemetry openTelemetry, MetadataNodeSizeStats nodeSizeStats, + int numSerDesThreads) { this.nodeSizeStats = nodeSizeStats == null ? new DummyMetadataNodeSizeStats() : nodeSizeStats; - this.executor = new ScheduledThreadPoolExecutor(1, - new DefaultThreadFactory( - StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName())); + final var namePrefix = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName + : getClass().getSimpleName(); + this.eventExecutor = Executors.newSingleThreadExecutor( + new DefaultThreadFactory(namePrefix + "-event")); + this.schedulerExecutor = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory(namePrefix + "-scheduler")); + this.serDesExecutor = OrderedExecutor.newBuilder() + .numThreads(numSerDesThreads) + .name(namePrefix + "-worker") + .build(); registerListener(this); this.childrenCache = Caffeine.newBuilder() @@ -249,7 +261,8 @@ public MetadataCache getMetadataCache(Class clazz, MetadataCacheConfig JavaType typeRef = TypeFactory.defaultInstance().constructSimpleType(clazz, null); String cacheName = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName(); MetadataCacheImpl metadataCache = - new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.executor); + new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.serDesExecutor, + this.schedulerExecutor); metadataCaches.add(metadataCache); return metadataCache; } @@ -258,7 +271,8 @@ public MetadataCache getMetadataCache(Class clazz, MetadataCacheConfig public MetadataCache getMetadataCache(TypeReference typeRef, MetadataCacheConfig cacheConfig) { String cacheName = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName(); MetadataCacheImpl metadataCache = - new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.executor); + new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.serDesExecutor, + this.schedulerExecutor); metadataCaches.add(metadataCache); return metadataCache; } @@ -268,7 +282,7 @@ public MetadataCache getMetadataCache(String cacheName, MetadataSerde MetadataCacheConfig cacheConfig) { MetadataCacheImpl metadataCache = new MetadataCacheImpl<>(cacheName, this, serde, cacheConfig, - this.executor); + this.serDesExecutor, this.schedulerExecutor); metadataCaches.add(metadataCache); return metadataCache; } @@ -348,7 +362,7 @@ protected CompletableFuture receivedNotification(Notification notification }); return null; - }, executor); + }, eventExecutor); } catch (RejectedExecutionException e) { return FutureUtil.failedFuture(e); } @@ -531,7 +545,7 @@ protected void receivedSessionEvent(SessionEvent event) { // Notice listeners. try { - executor.execute(() -> { + eventExecutor.execute(() -> { sessionListeners.forEach(l -> { try { l.accept(event); @@ -556,8 +570,9 @@ protected static CompletableFuture alreadyClosedFailedFuture() { @Override public void close() throws Exception { - executor.shutdownNow(); - executor.awaitTermination(10, TimeUnit.SECONDS); + MoreExecutors.shutdownAndAwaitTermination(serDesExecutor, 10, TimeUnit.SECONDS); + MoreExecutors.shutdownAndAwaitTermination(schedulerExecutor, 10, TimeUnit.SECONDS); + MoreExecutors.shutdownAndAwaitTermination(eventExecutor, 10, TimeUnit.SECONDS); this.metadataStoreStats.close(); } @@ -574,30 +589,30 @@ public void invalidateCaches(String...paths) { } } - /** - * Run the task in the executor thread and fail the future if the executor is shutting down. - */ - @VisibleForTesting - public void execute(Runnable task, CompletableFuture future) { + protected final void processEvent(Consumer eventProcessor, T event) { try { - executor.execute(task); - } catch (Throwable t) { - future.completeExceptionally(t); + eventExecutor.execute(() -> eventProcessor.accept(event)); + } catch (RejectedExecutionException e) { + log.warn("Rejected processing event {}", event); } } - /** - * Run the task in the executor thread and fail the future if the executor is shutting down. - */ - @VisibleForTesting - public void execute(Runnable task, Supplier>> futures) { + protected final void scheduleDelayedTask(long delay, TimeUnit unit, Runnable task) { + schedulerExecutor.schedule(task, delay, unit); + } + + protected final void safeExecuteCallback(Runnable task, Consumer exceptionHandler) { try { - executor.execute(task); - } catch (final Throwable t) { - futures.get().forEach(f -> f.completeExceptionally(t)); + eventExecutor.execute(task); + } catch (Throwable t) { + exceptionHandler.accept(t); } } + protected final void safeExecuteCallback(Runnable task, CompletableFuture future) { + safeExecuteCallback(task, future::completeExceptionally); + } + protected static String parent(String path) { int idx = path.lastIndexOf('/'); if (idx <= 0) { 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 3937fd712dc9f..e1311fccfe034 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 @@ -188,7 +188,7 @@ public void close() throws Exception { @Override protected CompletableFuture existsFromStore(String path) { return kv.get(ByteSequence.from(path, StandardCharsets.UTF_8), EXISTS_GET_OPTION) - .thenApplyAsync(gr -> gr.getCount() == 1, executor); + .thenApply(gr -> gr.getCount() == 1); } @Override @@ -204,9 +204,8 @@ protected CompletableFuture storePut(String path, byte[] data, Optional super.storePut(path + stat.getVersion(), data, optExpectedVersion, options), - executor); + .thenCompose( + stat -> super.storePut(path + stat.getVersion(), data, optExpectedVersion, options)); } } @@ -313,9 +312,7 @@ protected void batchOperation(List ops) { } } else { log.warn("Failed to commit: {}", cause.getMessage()); - executor.execute(() -> { - ops.forEach(o -> o.getFuture().completeExceptionally(ex)); - }); + ops.forEach(o -> o.getFuture().completeExceptionally(ex)); } return null; }); @@ -326,7 +323,7 @@ protected void batchOperation(List ops) { private void handleBatchOperationResult(TxnResponse txnResponse, List ops) { - executor.execute(() -> { + safeExecuteCallbacks(() -> { if (!txnResponse.isSucceeded()) { if (ops.size() > 1) { // Retry individually @@ -404,7 +401,7 @@ private void handleBatchOperationResult(TxnResponse txnResponse, } } } - }); + }, ops); } private synchronized CompletableFuture createLease(boolean retryOnFailure) { @@ -444,9 +441,7 @@ public void onCompleted() { if (retryOnFailure) { future.exceptionally(ex -> { log.warn("Failed to create Etcd lease. Retrying later", ex); - executor.schedule(() -> { - createLease(true); - }, 1, TimeUnit.SECONDS); + scheduleDelayedTask(1, TimeUnit.SECONDS, () -> createLease(true)); return null; }); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index 079cb3130e054..627304b2edc7d 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -79,7 +79,7 @@ private static class Value { public LocalMemoryMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats()); + metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); String name = metadataURL.substring(MEMORY_SCHEME_IDENTIFIER.length()); // Local means a private data set // update synchronizer and register sync listener 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 74bddda7454cb..08e5478ffcca1 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 @@ -210,7 +210,7 @@ static long toLong(byte[] bytes) { private RocksdbMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats()); + metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); this.metadataUrl = metadataURL; try { RocksDB.loadLibrary(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 5bf7e2272f022..f56d6c6941f1e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -119,7 +119,7 @@ public ZKMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConf private void processSessionWatcher(WatchedEvent event) { if (sessionWatcher != null) { - executor.execute(() -> sessionWatcher.process(event)); + processEvent(sessionWatcher::process, event); } } @@ -245,9 +245,8 @@ protected void batchOperation(List ops) { countsByType, totalSize, opsForLog); // Retry with the individual operations - executor.schedule(() -> { - ops.forEach(o -> batchOperation(Collections.singletonList(o))); - }, 100, TimeUnit.MILLISECONDS); + scheduleDelayedTask(100, TimeUnit.MILLISECONDS, + () -> ops.forEach(o -> batchOperation(Collections.singletonList(o)))); } else { MetadataStoreException e = getException(code, path); ops.forEach(o -> o.getFuture().completeExceptionally(e)); @@ -256,7 +255,7 @@ protected void batchOperation(List ops) { } // Trigger all the futures in the batch - execute(() -> { + safeExecuteCallbacks(() -> { for (int i = 0; i < ops.size(); i++) { OpResult opr = results.get(i); MetadataOp op = ops.get(i); @@ -278,7 +277,7 @@ protected void batchOperation(List ops) { "Operation type not supported in multi: " + op.getType())); } } - }, () -> ops.stream().map(MetadataOp::getFuture).collect(Collectors.toList())); + }, ops); }, null); } catch (Throwable t) { ops.forEach(o -> o.getFuture().completeExceptionally(new MetadataStoreException(t))); @@ -395,7 +394,7 @@ public CompletableFuture existsFromStore(String path) { try { zkc.exists(path, null, (rc, path1, ctx, stat) -> { - execute(() -> { + safeExecuteCallback(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(true); @@ -421,7 +420,7 @@ private void internalStoreDelete(OpDelete op) { try { zkc.delete(op.getPath(), expectedVersion, (rc, path1, ctx) -> { - execute(() -> { + safeExecuteCallback(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(null); @@ -446,7 +445,7 @@ private void internalStorePut(OpPut opPut) { CreateMode createMode = getCreateMode(opPut.getOptions()); asyncCreateFullPathOptimistic(zkc, opPut.getPath(), opPut.getData(), createMode, (rc, path1, ctx, name) -> { - execute(() -> { + safeExecuteCallback(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(new Stat(name, 0, 0, 0, createMode.isEphemeral(), true)); @@ -460,7 +459,7 @@ private void internalStorePut(OpPut opPut) { }); } else { zkc.setData(opPut.getPath(), opPut.getData(), expectedVersion, (rc, path1, ctx, stat) -> { - execute(() -> { + safeExecuteCallback(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(getStat(path1, stat)); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index a9319a50fec5c..30989a41bd167 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -18,16 +18,20 @@ */ package org.apache.pulsar.metadata.impl.batching; +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.util.concurrent.DefaultThreadFactory; import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +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.AtomicBoolean; import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataEventSynchronizer; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -38,6 +42,7 @@ import org.apache.pulsar.metadata.impl.stats.BatchMetadataStoreStats; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.MpscUnboundedArrayQueue; +import org.jspecify.annotations.Nullable; @Slf4j public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore { @@ -46,8 +51,6 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private final MessagePassingQueue readOps; private final MessagePassingQueue writeOps; - private final AtomicBoolean flushInProgress = new AtomicBoolean(false); - private final boolean enabled; private final int maxDelayMillis; protected final int maxOperations; @@ -55,9 +58,12 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private MetadataEventSynchronizer synchronizer; private final BatchMetadataStoreStats batchMetadataStoreStats; protected MetadataStoreBatchStrategy metadataStoreBatchStrategy; + @Nullable + private final ScheduledExecutorService flushExecutor; protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { - super(conf.getMetadataStoreName(), conf.getOpenTelemetry(), conf.getNodeSizeStats()); + super(conf.getMetadataStoreName(), conf.getOpenTelemetry(), conf.getNodeSizeStats(), + conf.getNumSerDesThreads()); this.enabled = conf.isBatchingEnabled(); this.maxDelayMillis = conf.getBatchingMaxDelayMillis(); @@ -67,18 +73,22 @@ protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { if (enabled) { readOps = new MpscUnboundedArrayQueue<>(10_000); writeOps = new MpscUnboundedArrayQueue<>(10_000); - scheduledTask = - executor.scheduleAtFixedRate(this::flush, maxDelayMillis, maxDelayMillis, TimeUnit.MILLISECONDS); + final var name = StringUtils.isNotBlank(conf.getMetadataStoreName()) ? conf.getMetadataStoreName() + : getClass().getSimpleName(); + flushExecutor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory( + name + "-batch-flusher")); + scheduledTask = flushExecutor.scheduleAtFixedRate(this::flush, maxDelayMillis, maxDelayMillis, + TimeUnit.MILLISECONDS); } else { scheduledTask = null; readOps = null; writeOps = null; + flushExecutor = null; } // update synchronizer and register sync listener updateMetadataEventSynchronizer(conf.getSynchronizer()); - this.batchMetadataStoreStats = - new BatchMetadataStoreStats(metadataStoreName, executor, conf.getOpenTelemetry()); + this.batchMetadataStoreStats = new BatchMetadataStoreStats(metadataStoreName); this.metadataStoreBatchStrategy = new DefaultMetadataStoreBatchStrategy(maxOperations, maxSize); } @@ -96,12 +106,13 @@ public void close() throws Exception { op.getFuture().completeExceptionally(ex); } scheduledTask.cancel(true); + MoreExecutors.shutdownAndAwaitTermination(flushExecutor, 10, TimeUnit.SECONDS); } super.close(); this.batchMetadataStoreStats.close(); } - private void flush() { + private synchronized void flush() { List currentBatch; if (!readOps.isEmpty()) { while (CollectionUtils.isNotEmpty(currentBatch = metadataStoreBatchStrategy.nextBatch(readOps))) { @@ -113,8 +124,6 @@ private void flush() { internalBatchOperation(currentBatch); } } - - flushInProgress.set(false); } @Override @@ -169,8 +178,8 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { internalBatchOperation(Collections.singletonList(op)); return; } - if (queue.size() > maxOperations && flushInProgress.compareAndSet(false, true)) { - executor.execute(this::flush); + if (queue.size() > maxOperations) { + flush(); } } else { internalBatchOperation(Collections.singletonList(op)); @@ -194,4 +203,8 @@ private void internalBatchOperation(List ops) { } protected abstract void batchOperation(List ops); + + protected final void safeExecuteCallbacks(Runnable runnable, List ops) { + safeExecuteCallback(runnable, t -> ops.forEach(op -> op.getFuture().completeExceptionally(t))); + } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index d055dd7da55fb..407a927bda4dc 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -61,7 +61,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { private Optional synchronizer; public OxiaMetadataStore(AsyncOxiaClient oxia, String identity) { - super("oxia-metadata", OpenTelemetry.noop(), null); + super("oxia-metadata", OpenTelemetry.noop(), null, 1); this.client = oxia; this.identity = identity; this.synchronizer = Optional.empty(); @@ -75,7 +75,7 @@ public OxiaMetadataStore( boolean enableSessionWatcher) throws Exception { super("oxia-metadata", Objects.requireNonNull(metadataStoreConfig).getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats()); + metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); var linger = metadataStoreConfig.getBatchingMaxDelayMillis(); if (!metadataStoreConfig.isBatchingEnabled()) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java index 9549a8df8f9f1..82cc15d8aafab 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java @@ -18,23 +18,13 @@ */ package org.apache.pulsar.metadata.impl.stats; -import io.opentelemetry.api.OpenTelemetry; -import io.opentelemetry.api.common.Attributes; -import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; -import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; public final class BatchMetadataStoreStats implements AutoCloseable { private static final double[] BUCKETS = new double[]{1, 5, 10, 20, 50, 100, 200, 500, 1000}; private static final String NAME = "name"; - private static final Gauge EXECUTOR_QUEUE_SIZE = Gauge - .build("pulsar_batch_metadata_store_executor_queue_size", "-") - .labelNames(NAME) - .register(); private static final Histogram OPS_WAITING = Histogram .build("pulsar_batch_metadata_store_queue_wait_time", "-") .unit("ms") @@ -54,46 +44,17 @@ public final class BatchMetadataStoreStats implements AutoCloseable { .register(); private final AtomicBoolean closed = new AtomicBoolean(false); - private final ThreadPoolExecutor executor; private final String metadataStoreName; private final Histogram.Child batchOpsWaitingChild; private final Histogram.Child batchExecuteTimeChild; private final Histogram.Child opsPerBatchChild; - public static final String EXECUTOR_QUEUE_SIZE_METRIC_NAME = "pulsar.broker.metadata.store.executor.queue.size"; - private final ObservableLongUpDownCounter batchMetadataStoreSizeCounter; - - public BatchMetadataStoreStats(String metadataStoreName, ExecutorService executor, OpenTelemetry openTelemetry) { - if (executor instanceof ThreadPoolExecutor tx) { - this.executor = tx; - } else { - this.executor = null; - } + public BatchMetadataStoreStats(String metadataStoreName) { this.metadataStoreName = metadataStoreName; - - EXECUTOR_QUEUE_SIZE.setChild(new Gauge.Child() { - @Override - public double get() { - return getQueueSize(); - } - }, metadataStoreName); - this.batchOpsWaitingChild = OPS_WAITING.labels(metadataStoreName); this.batchExecuteTimeChild = BATCH_EXECUTE_TIME.labels(metadataStoreName); this.opsPerBatchChild = OPS_PER_BATCH.labels(metadataStoreName); - - var meter = openTelemetry.getMeter("org.apache.pulsar"); - var attributes = Attributes.of(MetadataStoreStats.METADATA_STORE_NAME, metadataStoreName); - this.batchMetadataStoreSizeCounter = meter - .upDownCounterBuilder(EXECUTOR_QUEUE_SIZE_METRIC_NAME) - .setDescription("The number of batch operations in the metadata store executor queue") - .setUnit("{operation}") - .buildWithCallback(measurement -> measurement.record(getQueueSize(), attributes)); - } - - private int getQueueSize() { - return executor == null ? 0 : executor.getQueue().size(); } public void recordOpWaiting(long millis) { @@ -111,11 +72,9 @@ public void recordOpsInBatch(int ops) { @Override public void close() throws Exception { if (closed.compareAndSet(false, true)) { - EXECUTOR_QUEUE_SIZE.remove(this.metadataStoreName); OPS_WAITING.remove(this.metadataStoreName); BATCH_EXECUTE_TIME.remove(this.metadataStoreName); OPS_PER_BATCH.remove(metadataStoreName); - batchMetadataStoreSizeCounter.close(); } } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java index d42b2228346b8..0ae0b022a352d 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java @@ -96,7 +96,7 @@ public MetadataStore create(String metadataURL, MetadataStoreConfig metadataStor public static class MyMetadataStore extends AbstractMetadataStore { protected MyMetadataStore() { - super("custom", OpenTelemetry.noop(), null); + super("custom", OpenTelemetry.noop(), null, 1); } @Override From 64922f2784d6eb031888a6299a095a437eb5cd96 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 6 Feb 2026 00:53:25 +0800 Subject: [PATCH 25/53] [fix][test]Fix flaky ExtensibleLoadManagerImplTest_testGetMetrics (#25216) (cherry picked from commit 257d42ff9d4d7656a110d77013a89570c3feaa93) (cherry picked from commit a8eac911968458d3ae25fc23412bdb0d2f7f6a9e) --- .../loadbalance/extensions/ExtensibleLoadManagerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8b60e9c506659..d62e41450b808 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 @@ -1784,7 +1784,7 @@ public void testRoleChange() throws Exception { .get(3, TimeUnit.SECONDS); } - @Test + @Test(priority = Integer.MIN_VALUE) public void testGetMetrics() throws Exception { { ServiceConfiguration conf = getDefaultConf(); From c6333770eb5ecc9c0fa8a8678053b8f2f7d57bf8 Mon Sep 17 00:00:00 2001 From: Oneby Wang <44369297+oneby-wang@users.noreply.github.com> Date: Fri, 6 Feb 2026 00:58:59 +0800 Subject: [PATCH 26/53] [fix][broker] Fix ManagedCursorImpl.asyncDelete() method may lose previous async mark delete properties in race condition (#25165) (cherry picked from commit bea6f8ac48740b44c5d162bd4aa0ca851eb91ae5) (cherry picked from commit 4332a44f177d683cb3abede5c75d897e68cc6332) --- .../mledger/impl/ManagedCursorImpl.java | 5 +- .../mledger/impl/ManagedCursorTest.java | 100 ++++++++++++++++++ 2 files changed, 101 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 0b789fc2a243e..0a0042dafb28b 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 @@ -2676,10 +2676,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } try { - Map properties = lastMarkDeleteEntry != null ? lastMarkDeleteEntry.properties - : Collections.emptyMap(); - - internalAsyncMarkDelete(newMarkDeletePosition, properties, new MarkDeleteCallback() { + internalAsyncMarkDelete(newMarkDeletePosition, null, new MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { callback.deleteComplete(ctx); 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 d94c2ced7681b..0b9e3e6b08db8 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 @@ -5916,6 +5916,106 @@ public void testConcurrentRead() throws Exception { assertEquals(future1.get(2, TimeUnit.SECONDS).get(0).getData(), "msg".getBytes()); } + @Test(timeOut = 20000) + public void testAsyncMarkDeleteNeverLoseProperties() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(3); + config.setRetentionTime(20, TimeUnit.SECONDS); + config.setRetentionSizeInMB(5); + + @Cleanup ManagedLedgerImpl ledger = + (ManagedLedgerImpl) factory.open("testAsyncMarkDeleteNeverLoseProperties", config); + @Cleanup ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + int numMessages = 20; + List positions = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + Position pos = ledger.addEntry("entry-1".getBytes(Encoding)); + positions.add(pos); + } + + String propertyKey = "test-property"; + CountDownLatch latch = new CountDownLatch(numMessages); + for (int i = 0; i < numMessages; i++) { + Map properties = new HashMap<>(); + properties.put(propertyKey, (long) i); + cursor.asyncMarkDelete(positions.get(i), properties, new MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + latch.countDown(); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + fail("Mark delete should succeed"); + } + }, null); + } + + latch.await(); + + int lastIndex = numMessages - 1; + assertEquals(cursor.getMarkDeletedPosition(), positions.get(lastIndex)); + Map properties = cursor.getProperties(); + assertEquals(properties.size(), 1); + assertEquals(properties.get(propertyKey), lastIndex); + } + + @Test(timeOut = 20000) + public void testAsyncDeleteNeverLoseMarkDeleteProperties() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(11); + + @Cleanup ManagedLedgerImpl ledger = + (ManagedLedgerImpl) factory.open("testAsyncDeleteNeverLoseMarkDeleteProperty", config); + @Cleanup ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + int numMessages = 10; + List positions = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + Position pos = ledger.addEntry("entry-1".getBytes(Encoding)); + positions.add(pos); + } + + String propertyKey = "test-property"; + CountDownLatch latch = new CountDownLatch(numMessages); + for (int i = 0; i < numMessages - 1; i++) { + Map properties = new HashMap<>(); + properties.put(propertyKey, (long) i); + cursor.asyncMarkDelete(positions.get(i), properties, new MarkDeleteCallback() { + @Override + public void markDeleteComplete(Object ctx) { + latch.countDown(); + } + + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + fail("Mark delete should succeed"); + } + }, null); + } + + int lastIndex = numMessages - 1; + cursor.asyncDelete(positions.get(lastIndex), new DeleteCallback() { + @Override + public void deleteComplete(Object ctx) { + latch.countDown(); + } + + @Override + public void deleteFailed(ManagedLedgerException exception, Object ctx) { + fail("Delete should succeed"); + } + }, null); + + latch.await(); + + assertEquals(cursor.getMarkDeletedPosition(), positions.get(lastIndex)); + Map properties = cursor.getProperties(); + assertEquals(properties.size(), 1); + assertEquals(properties.get(propertyKey), lastIndex - 1); + } + class TestPulsarMockBookKeeper extends PulsarMockBookKeeper { Map ledgerErrors = new HashMap<>(); From bfb8edca022258f270da64889fb750829f0fd0e7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 6 Feb 2026 01:01:16 +0800 Subject: [PATCH 27/53] [fix][broker]Fix ledgerHandle failed to read by using new BK API (#25199) (cherry picked from commit 6d51f8883206fd81840faddf4cec840f6969b137) (cherry picked from commit 1631fedda57c55342352e7a4cc4da31c4508d772) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 +- .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 4 ++-- .../bookkeeper/mledger/impl/ManagedCursorTest.java | 9 +++++++++ .../mledger/impl/ManagedLedgerFactoryShutdownTest.java | 6 ++++++ .../delayed/bucket/BookkeeperBucketSnapshotStorage.java | 2 +- .../broker/service/schema/BookkeeperSchemaStorage.java | 2 +- .../org/apache/pulsar/compaction/CompactedTopicImpl.java | 2 +- 7 files changed, 21 insertions(+), 6 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 0a0042dafb28b..790c81cadc9ba 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 @@ -685,7 +685,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac }; try { bookkeeper.asyncOpenLedger(ledgerId, digestType, getConfig().getPassword(), openCallback, - null); + null, true); } catch (Throwable t) { log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", ledger.getName(), ledgerId, name, t); 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 91ae69e61167d..ff801eaeaa591 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 @@ -472,7 +472,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { log.debug("[{}] Opening ledger {}", name, id); } mbean.startDataLedgerOpenOp(); - bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null); + bookKeeper.asyncOpenLedger(id, digestType, config.getPassword(), opencb, null, true); } else { initializeBookKeeper(callback); } @@ -1809,7 +1809,7 @@ synchronized void addEntryFailedDueToConcurrentlyModified(final LedgerHandle cur handleBadVersion(new BadVersionException("the current ledger " + currentLedger.getId() + " was concurrent modified by a other bookie client. The error code is: " + errorCode)); } - }, null); + }, null, true); } synchronized void ledgerClosed(final LedgerHandle lh) { 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 0b9e3e6b08db8..6b37c4a5c18aa 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 @@ -6035,6 +6035,15 @@ public void asyncOpenLedger(final long lId, final DigestType digestType, final b super.asyncOpenLedger(lId, digestType, passwd, cb, ctx); } } + + public void asyncOpenLedger(final long lId, final DigestType digestType, final byte[] passwd, + final OpenCallback cb, final Object ctx, boolean keepMetadataUpdate) { + if (ledgerErrors.containsKey(lId)) { + cb.openComplete(ledgerErrors.get(lId), null, ctx); + } else { + super.asyncOpenLedger(lId, digestType, passwd, cb, ctx, keepMetadataUpdate); + } + } } private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java index ecc3423e292e9..95f0a6b8c775e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.mledger.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.anyString; @@ -137,6 +138,11 @@ private void setup() { cb.openComplete(0, ledgerHandle, inv.getArgument(4, Object.class)); return null; }).when(bookKeeper).asyncOpenLedger(anyLong(), any(), any(), any(), any()); + doAnswer(inv -> { + AsyncCallback.OpenCallback cb = inv.getArgument(3, AsyncCallback.OpenCallback.class); + cb.openComplete(0, ledgerHandle, inv.getArgument(4, Object.class)); + return null; + }).when(bookKeeper).asyncOpenLedger(anyLong(), any(), any(), any(), any(), anyBoolean()); doAnswer(inv -> { AsyncCallback.CreateCallback cb = inv.getArgument(5, AsyncCallback.CreateCallback.class); cb.createComplete(0, newLedgerHandle, inv.getArgument(6, Object.class)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java index ba37092e88d9b..fa7408d7e1574 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java @@ -213,7 +213,7 @@ private CompletableFuture openLedger(Long ledgerId) { } else { future.complete(handle); } - }, null + }, null, true ); return future; } 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 b931239a32cea..e38bf48f1fdb1 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 @@ -630,7 +630,7 @@ private CompletableFuture openLedger(Long ledgerId) { } else { future.complete(handle); } - }, null + }, null, true ); return future; } 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 c1469b407cf76..160c1525480a3 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 @@ -184,7 +184,7 @@ private static CompletableFuture openCompactedLedger(Book } else { promise.complete(ledger); } - }, null); + }, null, true); return promise.thenApply((ledger) -> new CompactedTopicContext( ledger, createCache(ledger, DEFAULT_MAX_CACHE_SIZE))); } From 28f5af640985aa3aeddc20240d7eab5c043568d9 Mon Sep 17 00:00:00 2001 From: Malla Sandeep Date: Fri, 6 Feb 2026 00:51:54 +0530 Subject: [PATCH 28/53] [fix][client] Fix producer synchronous retry handling in failPendingMessages method (#25207) (cherry picked from commit 611efe4a77a4cc9ec1875d4ee5fd29916d3f75c7) (cherry picked from commit 30ae8fba3f930c1aeb5baae2e5fad2db4b3ea734) --- .../client/impl/ProducerSyncRetryTest.java | 99 +++++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 20 ++-- .../pulsar/client/impl/ProducerImplTest.java | 77 +++++++++++++++ 3 files changed, 189 insertions(+), 7 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSyncRetryTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSyncRetryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSyncRetryTest.java new file mode 100644 index 0000000000000..359db97fc78fc --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerSyncRetryTest.java @@ -0,0 +1,99 @@ +/* + * 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.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-impl") +public class ProducerSyncRetryTest extends ProducerConsumerBase { + + @Override + @BeforeMethod + public void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + @AfterMethod(alwaysRun = true) + public void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30000) + public void testProducerSyncRetryAfterTimeout() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp"); + @Cleanup + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .sendTimeout(1, TimeUnit.MILLISECONDS) // force timeout + .create(); + + // To make sure first message is timed out + this.stopBroker(); + + // First message will get timed out, then be retried with same payload + ByteBuffer payload = ByteBuffer.wrap(new byte[0]); + MessageMetadata messageMetadata = new MessageMetadata(); + messageMetadata.setUncompressedSize(1); + MessageImpl message = MessageImpl.create(messageMetadata, payload, Schema.BYTES, topic); + + MessageMetadata retryMessageMetadata = new MessageMetadata(); + retryMessageMetadata.setUncompressedSize(1); + MessageImpl retryMessage = MessageImpl.create(retryMessageMetadata, payload, Schema.BYTES, topic); + + // First send is expected to fail + CompletableFuture firstSend = producer.sendAsync(message); + producer.triggerSendTimer(); + + // Waits until firstSend returns timeout exception + CompletableFuture retrySend = + firstSend.handle((msgId, ex) -> { + assertNotNull(ex, "First send must timeout"); + assertTrue(ex instanceof PulsarClientException.TimeoutException); + try { + // Retry should succeed + this.startBroker(); + } catch (Exception e) { + throw new RuntimeException(e); + } + producer.conf.setSendTimeoutMs(10000); + return producer.sendAsync(retryMessage); + }).thenCompose(f -> f); + + // Wait until retry completes successfully + MessageId retryMessageId = retrySend.join(); + assertNotNull(retryMessageId); + } +} 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 808e896ea684b..b070aab79ea5b 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 @@ -1801,11 +1801,12 @@ public void clear() { messagesCount.set(0); } - public void remove() { + public OpSendMsg remove() { OpSendMsg op = delegate.remove(); if (op != null) { messagesCount.addAndGet(-op.numMessagesInBatch); } + return op; } public OpSendMsg peek() { @@ -2276,14 +2277,20 @@ public void run(Timeout timeout) throws Exception { } /** - * This fails and clears the pending messages with the given exception. This method should be called from within the - * ProducerImpl object mutex. + * This fails the pending messages at the start of the call, without dropping newly enqueued + * retry messages. This method should be called from within the ProducerImpl object mutex. */ - private synchronized void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { + @VisibleForTesting + synchronized void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { if (cnx == null) { final AtomicInteger releaseCount = new AtomicInteger(); final boolean batchMessagingEnabled = isBatchMessagingEnabled(); - pendingMessages.forEach(op -> { + // Track message count to fail so that newly added messages by synchronous retries + // triggered by op.sendComplete(ex); don't get removed + int pendingMessagesToFailCount = pendingMessages.size(); + + for (int i = 0; i < pendingMessagesToFailCount; i++) { + OpSendMsg op = pendingMessages.remove(); releaseCount.addAndGet(batchMessagingEnabled ? op.numMessagesInBatch : 1); try { // Need to protect ourselves from any exception being thrown in the future handler from the @@ -2303,9 +2310,8 @@ private synchronized void failPendingMessages(ClientCnx cnx, PulsarClientExcepti client.getMemoryLimitController().releaseMemory(op.uncompressedSize); ReferenceCountUtil.safeRelease(op.cmd); op.recycle(); - }); + } - pendingMessages.clear(); semaphoreRelease(releaseCount.get()); if (batchMessagingEnabled) { failPendingBatchMessages(ex); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index 40b76f37cc0c6..0fe177b775cbd 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -26,9 +26,12 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import java.nio.ByteBuffer; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.protocol.ByteBufPair; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -67,4 +70,78 @@ public void testPopulateMessageSchema() { assertTrue(producer.populateMessageSchema(msg, null)); verify(msg).setSchemaState(MessageImpl.SchemaState.Ready); } + + @Test + public void testFailPendingMessagesSyncRetry() + throws Exception { + ProducerImpl producer = + Mockito.mock(ProducerImpl.class, Mockito.CALLS_REAL_METHODS); + // Disable batching + Mockito.doReturn(false) + .when(producer) + .isBatchMessagingEnabled(); + + // Stub semaphore release (not under test) + Mockito.doNothing() + .when(producer) + .semaphoreRelease(Mockito.anyInt()); + + // Stub client cleanup path (not under test) + PulsarClientImpl client = Mockito.mock(PulsarClientImpl.class); + Mockito.when(client.getMemoryLimitController()) + .thenReturn(Mockito.mock(MemoryLimitController.class)); + FieldUtils.writeField(producer, "client", client, true); + + // Real pending queue + ProducerImpl.OpSendMsgQueue pendingQueue = new ProducerImpl.OpSendMsgQueue(); + FieldUtils.writeField(producer, "pendingMessages", pendingQueue, true); + + // OpSendMsg that retries reentrantly + MessageImpl msg = Mockito.mock(MessageImpl.class); + Mockito.when(msg.getUncompressedSize()).thenReturn(10); + ProducerImpl.OpSendMsg op = ProducerImpl.OpSendMsg.create( + Mockito.mock(LatencyHistogram.class), + msg, + Mockito.mock(ByteBufPair.class), + 1L, + Mockito.mock(SendCallback.class) + ); + op.totalChunks = 1; + op.chunkId = 0; + op.numMessagesInBatch = 1; + + MessageImpl retryMsg = Mockito.mock(MessageImpl.class); + Mockito.when(retryMsg.getUncompressedSize()).thenReturn(10); + + // Override sendComplete to Reentrant retry via spy + ProducerImpl.OpSendMsg firstSpy = Mockito.spy(op); + Mockito.doAnswer(invocation -> { + // Reentrant retry during callback + ProducerImpl.OpSendMsg retryOp = ProducerImpl.OpSendMsg.create( + Mockito.mock(LatencyHistogram.class), + retryMsg, + Mockito.mock(ByteBufPair.class), + 2L, + Mockito.mock(SendCallback.class) + ); + retryOp.totalChunks = 1; + retryOp.chunkId = 0; + retryOp.numMessagesInBatch = 1; + pendingQueue.add(retryOp); + return null; + }).when(firstSpy).sendComplete(Mockito.any()); + Mockito.doNothing() + .when(firstSpy) + .recycle(); + + // Seed initial pending message + pendingQueue.add(firstSpy); + + // Invoke failPendingMessages(null, ex) + producer.failPendingMessages(null, new PulsarClientException.TimeoutException("timeout")); + assertEquals(producer.getPendingQueueSize(), 1, + "Retry Op should exist in the pending Queue"); + assertEquals(pendingQueue.peek().sequenceId, 2L, + "Retry Op SequenceId should match with the one in pendingQueue"); + } } From 564147ccd2fe6344f8e6a57138318888bdb9db02 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 6 Feb 2026 10:31:56 +0200 Subject: [PATCH 29/53] [fix][broker] Prevent missed topic changes in topic watchers and schedule periodic refresh with patternAutoDiscoveryPeriod interval (#25188) (cherry picked from commit 2e06cc03570d69f8a0df0271fd5178438ea06fff) (cherry picked from commit ba2a2303efe53372a5934c0491b2311fc4f0146b) --- .../broker/resources/PulsarResources.java | 8 +- .../broker/resources/TopicListener.java | 29 ++ .../broker/resources/TopicResources.java | 83 +++- .../broker/resources/TopicResourcesTest.java | 103 +++-- .../broker/service/TopicListService.java | 375 ++++++++++++------ .../pulsar/broker/service/ServerCnxTest.java | 5 +- .../broker/service/TopicListServiceTest.java | 363 ++++++++++++++++- .../broker/service/TopicListWatcherTest.java | 8 +- .../NonStartableTestPulsarService.java | 2 +- ...cherBackPressureMultipleConsumersTest.java | 17 +- .../AutoCloseUselessClientConSupports.java | 9 + ...essClientConTopicsPatternConsumerTest.java | 3 + .../impl/PatternTopicsConsumerImplTest.java | 195 +++++---- .../apache/pulsar/client/impl/ClientCnx.java | 19 + .../impl/PatternConsumerUpdateQueue.java | 176 +++++--- .../impl/PatternMultiTopicsConsumerImpl.java | 216 +++++++--- .../pulsar/client/impl/PulsarClientImpl.java | 9 +- .../pulsar/client/impl/TopicListWatcher.java | 75 +++- .../client/impl/ClientTestFixtures.java | 16 +- .../impl/PatternConsumerUpdateQueueTest.java | 74 ++-- .../PatternMultiTopicsConsumerImplTest.java | 268 ++++++++++++- .../client/impl/TopicListWatcherTest.java | 81 +++- .../pulsar/common/protocol/Commands.java | 2 + .../pulsar/common/topics/TopicList.java | 2 +- pulsar-common/src/main/proto/PulsarApi.proto | 6 +- .../AutoCloseUselessClientConProxyTest.java | 5 + 26 files changed, 1675 insertions(+), 474 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicListener.java 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 cc64eeb52f6eb..7b89fe69194c3 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 @@ -52,20 +52,20 @@ public class PulsarResources { @Getter private final LoadBalanceResources loadBalanceResources; @Getter - private final Optional localMetadataStore; + private final Optional localMetadataStore; @Getter private final Optional configurationMetadataStore; - public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore) { + public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStore configurationMetadataStore) { this(localMetadataStore, configurationMetadataStore, DEFAULT_OPERATION_TIMEOUT_SEC); } - public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, + public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStore configurationMetadataStore, int operationTimeoutSec) { this(localMetadataStore, configurationMetadataStore, operationTimeoutSec, ForkJoinPool.commonPool()); } - public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, + public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStore configurationMetadataStore, int operationTimeoutSec, Executor executor) { if (configurationMetadataStore != null) { tenantResources = new TenantResources(configurationMetadataStore, operationTimeoutSec); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicListener.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicListener.java new file mode 100644 index 0000000000000..1d862067f15de --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicListener.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.pulsar.broker.resources; + +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.extended.SessionEvent; + +public interface TopicListener { + NamespaceName getNamespaceName(); + void onTopicEvent(String topicName, NotificationType notificationType); + void onSessionEvent(SessionEvent sessionEvent); +} 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 de7596a763889..91e84096d15e2 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 @@ -30,24 +30,27 @@ import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; 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.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.extended.SessionEvent; @Slf4j public class TopicResources { private static final String MANAGED_LEDGER_PATH = "/managed-ledgers"; - private final MetadataStore store; + private final MetadataStoreExtended store; - private final Map, Pattern> topicListeners; + private final Map topicListeners; - public TopicResources(MetadataStore store) { + public TopicResources(MetadataStoreExtended store) { this.store = store; topicListeners = new ConcurrentHashMap<>(); store.registerListener(this::handleNotification); + store.registerSessionListener(this::handleSessionEvent); } /*** @@ -111,13 +114,15 @@ void handleNotification(Notification notification) { if (notification.getPath().startsWith(MANAGED_LEDGER_PATH) && (notification.getType() == NotificationType.Created || notification.getType() == NotificationType.Deleted)) { - for (Map.Entry, Pattern> entry : + for (Map.Entry entry : topicListeners.entrySet()) { Matcher matcher = entry.getValue().matcher(notification.getPath()); if (matcher.matches()) { TopicName topicName = TopicName.get( matcher.group(2), NamespaceName.get(matcher.group(1)), decode(matcher.group(3))); - entry.getKey().accept(topicName.toString(), notification.getType()); + if (!SystemTopicNames.isSystemTopic(topicName)) { + entry.getKey().onTopicEvent(topicName.toString(), notification.getType()); + } } } } @@ -128,13 +133,75 @@ Pattern namespaceNameToTopicNamePattern(NamespaceName namespaceName) { + TopicDomain.persistent + ")/(" + "[^/]+)"); } + public void registerPersistentTopicListener(TopicListener listener) { + topicListeners.put(listener, namespaceNameToTopicNamePattern(listener.getNamespaceName())); + } + + public void deregisterPersistentTopicListener(TopicListener listener) { + topicListeners.remove(listener); + } + + private void handleSessionEvent(SessionEvent sessionEvent) { + topicListeners.keySet().forEach(listener -> { + try { + listener.onSessionEvent(sessionEvent); + } catch (Exception e) { + log.warn("Failed to handle session event {} for listener {}", sessionEvent, listener, e); + } + }); + } + + @Deprecated public void registerPersistentTopicListener( NamespaceName namespaceName, BiConsumer listener) { - topicListeners.put(listener, namespaceNameToTopicNamePattern(namespaceName)); + topicListeners.put(new BiConsumerTopicListener(listener, namespaceName), + namespaceNameToTopicNamePattern(namespaceName)); } + @Deprecated public void deregisterPersistentTopicListener(BiConsumer listener) { - topicListeners.remove(listener); + topicListeners.remove(new BiConsumerTopicListener(listener, null)); } + // for backwards compatibility with broker plugins that could be using the BiConsumer based methods + @Deprecated + static class BiConsumerTopicListener implements TopicListener { + private final BiConsumer listener; + private final NamespaceName namespaceName; + + BiConsumerTopicListener(BiConsumer listener, NamespaceName namespaceName) { + this.listener = listener; + this.namespaceName = namespaceName; + } + + @Override + public NamespaceName getNamespaceName() { + return namespaceName; + } + + @Override + public void onTopicEvent(String topicName, NotificationType notificationType) { + listener.accept(topicName, notificationType); + } + + @Override + public void onSessionEvent(SessionEvent sessionEvent) { + // ignore + } + + @Override + public String toString() { + return "BiConsumerTopicListener [listener=" + listener + ", namespaceName=" + namespaceName + "]"; + } + + @Override + public int hashCode() { + return listener.hashCode(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof BiConsumerTopicListener && listener.equals(((BiConsumerTopicListener) obj).listener); + } + } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/TopicResourcesTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/TopicResourcesTest.java index 48092e2a830ba..13c8f8d01c019 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/TopicResourcesTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/TopicResourcesTest.java @@ -21,24 +21,24 @@ import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; import java.util.function.BiConsumer; import org.apache.pulsar.common.naming.NamespaceName; -import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class TopicResourcesTest { - private MetadataStore metadataStore; + private MetadataStoreExtended metadataStore; private TopicResources topicResources; @BeforeMethod public void setup() { - metadataStore = mock(MetadataStore.class); + metadataStore = mock(MetadataStoreExtended.class); topicResources = new TopicResources(metadataStore); } @@ -49,65 +49,87 @@ public void testConstructorRegistersAsListener() { @Test public void testListenerInvokedWhenTopicCreated() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic")); - verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Created); + verify(listener).onTopicEvent("persistent://tenant/namespace/topic", NotificationType.Created); } @Test public void testListenerInvokedWhenTopicV1Created() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/cluster/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/cluster/namespace")); + topicResources.registerPersistentTopicListener(listener); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/cluster/namespace/persistent/topic")); - verify(listener).accept("persistent://tenant/cluster/namespace/topic", NotificationType.Created); + verify(listener).onTopicEvent("persistent://tenant/cluster/namespace/topic", NotificationType.Created); } @Test public void testListenerInvokedWhenTopicDeleted() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); topicResources.handleNotification(new Notification(NotificationType.Deleted, "/managed-ledgers/tenant/namespace/persistent/topic")); - verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Deleted); + verify(listener).onTopicEvent("persistent://tenant/namespace/topic", NotificationType.Deleted); } @Test public void testListenerNotInvokedWhenSubscriptionCreated() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic/subscription")); - verifyNoInteractions(listener); + verifyNoMoreInteractions(listener); } @Test public void testListenerNotInvokedWhenTopicCreatedInOtherNamespace() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace2/persistent/topic")); - verifyNoInteractions(listener); + verifyNoMoreInteractions(listener); } @Test public void testListenerNotInvokedWhenTopicModified() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Modified, "/managed-ledgers/tenant/namespace/persistent/topic")); - verifyNoInteractions(listener); + verifyNoMoreInteractions(listener); + } + + @Test + public void testListenerNotInvokedForSystemTopicChanges() { + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); + topicResources.handleNotification(new Notification(NotificationType.Created, + "/managed-ledgers/tenant/namespace/persistent/__change_events")); + verifyNoMoreInteractions(listener); } @Test public void testListenerNotInvokedAfterDeregistered() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic")); - verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Created); + verify(listener).onTopicEvent("persistent://tenant/namespace/topic", NotificationType.Created); topicResources.deregisterPersistentTopicListener(listener); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic2")); @@ -116,22 +138,39 @@ public void testListenerNotInvokedAfterDeregistered() { @Test public void testListenerInvokedWithDecodedTopicName() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/namespace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic%3Atest")); - verify(listener).accept("persistent://tenant/namespace/topic:test", NotificationType.Created); + verify(listener).onTopicEvent("persistent://tenant/namespace/topic:test", NotificationType.Created); } @Test public void testNamespaceContainsDotsShouldntMatchAny() { - BiConsumer listener = mock(BiConsumer.class); - topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/name.pace"), listener); + TopicListener listener = mock(TopicListener.class); + when(listener.getNamespaceName()).thenReturn(NamespaceName.get("tenant/name.pace")); + topicResources.registerPersistentTopicListener(listener); + verify(listener).getNamespaceName(); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic")); - verifyNoInteractions(listener); + verifyNoMoreInteractions(listener); topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/name.pace/persistent/topic")); - verify(listener).accept("persistent://tenant/name.pace/topic", NotificationType.Created); + verify(listener).onTopicEvent("persistent://tenant/name.pace/topic", NotificationType.Created); + } + + @Test + public void testBiConsumerListenerNotInvokedAfterDeregistered() { + BiConsumer listener = mock(BiConsumer.class); + topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener); + topicResources.handleNotification(new Notification(NotificationType.Created, + "/managed-ledgers/tenant/namespace/persistent/topic")); + verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Created); + topicResources.deregisterPersistentTopicListener(listener); + topicResources.handleNotification(new Notification(NotificationType.Created, + "/managed-ledgers/tenant/namespace/persistent/topic2")); + verifyNoMoreInteractions(listener); } } 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 ec0f9b73e7e8f..262c734ea9555 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 @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashSet; +import java.util.LinkedList; import java.util.List; import java.util.Set; import java.util.concurrent.BlockingDeque; @@ -35,13 +36,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +import lombok.Getter; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.resources.TopicListener; import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.topiclistlimit.TopicListMemoryLimiter; import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; @@ -59,11 +62,12 @@ import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TopicListService { - public static class TopicListWatcher implements BiConsumer { + public static class TopicListWatcher implements TopicListener { // upper bound for buffered topic list updates private static final int DEFAULT_TOPIC_LIST_UPDATE_MAX_QUEUE_SIZE = 10000; /** Topic names which are matching, the topic name contains the partition suffix. **/ @@ -77,7 +81,12 @@ public static class TopicListWatcher implements BiConsumer sendTopicListUpdateTasks; + @Getter(onMethod_ = @VisibleForTesting) private boolean updatingTopics; + private List matchingTopicsBeforeDisconnected; + private boolean disconnected; + private List updateCallbacks = new LinkedList<>(); + private boolean updatingWhileDisconnected; public TopicListWatcher(TopicListService topicListService, long id, NamespaceName namespace, TopicsPattern topicsPattern, List topics, @@ -103,8 +112,8 @@ public synchronized Collection getMatchingTopics() { * @param topicName topic name which contains partition suffix. */ @Override - public synchronized void accept(String topicName, NotificationType notificationType) { - if (closed || updatingTopics) { + public synchronized void onTopicEvent(String topicName, NotificationType notificationType) { + if (closed) { return; } String partitionedTopicName = TopicName.get(topicName).getPartitionedTopicName(); @@ -117,7 +126,7 @@ public synchronized void accept(String topicName, NotificationType notificationT if (matchingTopics.remove(topicName)) { deletedTopics = Collections.singletonList(topicName); } - } else if (matchingTopics.add(topicName)) { + } else if (notificationType == NotificationType.Created && matchingTopics.add(topicName)) { newTopics = Collections.singletonList(topicName); } if (!newTopics.isEmpty() || !deletedTopics.isEmpty()) { @@ -129,7 +138,7 @@ public synchronized void accept(String topicName, NotificationType notificationT // sends updates one-by-one so that ordering is retained private synchronized void sendTopicListUpdate(String hash, List deletedTopics, List newTopics) { - if (closed || updatingTopics) { + if (closed) { return; } Runnable task = () -> topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics, @@ -140,12 +149,9 @@ private synchronized void sendTopicListUpdate(String hash, List deletedT } else { // if sendTopicListSuccess hasn't completed, add to a queue to be executed after it completes if (!sendTopicListUpdateTasks.offer(task)) { - log.warn("Update queue was full for watcher id {} matching {}. Performing full refresh.", id, - topicsPattern.inputPattern()); - if (!updatingTopics) { - updatingTopics = true; - sendTopicListUpdateTasks.clear(); - matchingTopics.clear(); + if (prepareUpdateTopics(null)) { + log.warn("Update queue was full for watcher id {} matching {}. Performing full refresh.", id, + topicsPattern.inputPattern()); executor.execute(() -> topicListService.updateTopicListWatcher(this)); } } @@ -171,12 +177,112 @@ synchronized void sendingCompleted() { public synchronized void close() { closed = true; sendTopicListUpdateTasks.clear(); + updateCallbacks.clear(); + } + + /** + * Returns true if the topic list update is prepared for execution. It is expected that the caller initiates + * the update. The callback is registered to be executed after the update, either existing or upcoming is + * completed. + * @param afterUpdateCompletionCallback callback to be executed after the update is completed. + * @return true if an existing update wasn't ongoing and a new update is prepared for execution. + */ + synchronized boolean prepareUpdateTopics(Runnable afterUpdateCompletionCallback) { + if (!updatingTopics) { + updatingTopics = true; + sendingInProgress = true; + sendTopicListUpdateTasks.clear(); + matchingTopics.clear(); + if (afterUpdateCompletionCallback != null) { + updateCallbacks.add(afterUpdateCompletionCallback); + } + return true; + } else { + if (afterUpdateCompletionCallback != null) { + updateCallbacks.add(afterUpdateCompletionCallback); + } + return false; + } } synchronized void updateTopics(List topics) { + if (closed) { + return; + } matchingTopics.clear(); TopicList.filterTopicsToStream(topics, topicsPattern).forEach(matchingTopics::add); updatingTopics = false; + if (disconnected) { + handleNewAndDeletedTopicsWhileDisconnected(); + matchingTopicsBeforeDisconnected = null; + disconnected = false; + } + for (Runnable callback : updateCallbacks) { + try { + callback.run(); + } catch (Exception e) { + log.warn("Error executing topic list update callback: {}", callback, e); + } + } + updateCallbacks.clear(); + sendingCompleted(); + } + + private synchronized void handleNewAndDeletedTopicsWhileDisconnected() { + if (matchingTopicsBeforeDisconnected == null) { + return; + } + List newTopics = new ArrayList<>(); + List deletedTopics = new ArrayList<>(); + Set remainingTopics = new HashSet<>(matchingTopics); + for (String topic : matchingTopicsBeforeDisconnected) { + if (!remainingTopics.remove(topic)) { + deletedTopics.add(topic); + } + } + newTopics.addAll(remainingTopics); + if (!newTopics.isEmpty() || !deletedTopics.isEmpty()) { + String hash = TopicList.calculateHash(matchingTopics); + sendTopicListUpdate(hash, deletedTopics, newTopics); + } + } + + @Override + public NamespaceName getNamespaceName() { + return namespace; + } + + @Override + public synchronized void onSessionEvent(SessionEvent event) { + switch (event) { + case SessionReestablished: + case Reconnected: + executor.execute(() -> { + synchronized (this) { + // ensure that only one update is triggered when connection is being lost and reconnected + // before the updating is complete. The updatingWhileDisconnected flag is reseted after + // the update completes. + if (!updatingWhileDisconnected) { + updatingWhileDisconnected = true; + CompletableFuture future = topicListService.updateTopicListWatcher(this); + future.whenComplete((__, ___) -> { + synchronized (this) { + updatingWhileDisconnected = false; + } + }); + } + } + }); + break; + case SessionLost: + case ConnectionLost: + if (!disconnected) { + disconnected = true; + matchingTopicsBeforeDisconnected = new ArrayList<>(matchingTopics); + prepareUpdateTopics(null); + } + break; + } } } @@ -266,9 +372,27 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo CompletableFuture existingWatcherFuture = watchers.putIfAbsent(watcherId, watcherFuture); if (existingWatcherFuture != null) { - log.info("[{}] Watcher with the same watcherId={} is already created.", connection, watcherId); + if (log.isDebugEnabled()) { + log.debug("[{}] Watcher with the same watcherId={} is already created. Refreshing.", connection, + watcherId); + } // use the existing watcher if it's already created - watcherFuture = existingWatcherFuture; + watcherFuture = existingWatcherFuture.thenCompose(watcher -> { + CompletableFuture future = new CompletableFuture<>(); + Runnable callback = () -> future.complete(watcher); + // trigger a new update unless an update is already ongoing. Register the callback to complete + // when the update completes. + if (watcher.prepareUpdateTopics(callback)) { + updateTopicListWatcher(watcher) + // run the callback also in failure cases + // prepareUpdateTopics handles it for success cases + .exceptionally(ex -> { + callback.run(); + return null; + }); + } + return future; + }); } else { initializeTopicsListWatcher(watcherFuture, namespaceName, watcherId, topicsPattern); } @@ -290,8 +414,8 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo lookupSemaphore.release(); }) .exceptionally(ex -> { - log.warn("[{}] Error WatchTopicList for namespace [//{}] by {}", - connection.toString(), namespaceName, requestId); + log.warn("[{}] Error WatchTopicList for namespace [//{}] by {}: {}", + connection.toString(), namespaceName, requestId, ex.getMessage()); connection.getCommandSender().sendErrorResponse(requestId, BrokerServiceException.getClientErrorCode( new BrokerServiceException.ServerMetadataException(ex)), ex.getMessage()); @@ -327,130 +451,139 @@ private void sendTopicListSuccessWithPermitAcquiringRetries(long watcherId, long */ public void initializeTopicsListWatcher(CompletableFuture watcherFuture, NamespaceName namespace, long watcherId, TopicsPattern topicsPattern) { - BooleanSupplier isPermitRequestCancelled = () -> !connection.isActive() || !watchers.containsKey(watcherId); - if (isPermitRequestCancelled.getAsBoolean()) { - return; - } - TopicListSizeResultCache.ResultHolder listSizeHolder = pulsar.getBrokerService().getTopicListSizeResultCache() - .getTopicListSize(namespace.toString(), CommandGetTopicsOfNamespace.Mode.PERSISTENT); - AsyncDualMemoryLimiter maxTopicListInFlightLimiter = pulsar.getBrokerService().getMaxTopicListInFlightLimiter(); - - listSizeHolder.getSizeAsync().thenCompose(initialSize -> { - // use heap size limiter to avoid broker getting overwhelmed by a lot of concurrent topic list requests - return maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { - AtomicReference watcherRef = new AtomicReference<>(); - return namespaceService.getListOfPersistentTopics(namespace).thenCompose(topics -> { - long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); - listSizeHolder.updateSize(actualSize); - // register watcher immediately so that we don't lose events - TopicListWatcher watcher = - new TopicListWatcher(this, watcherId, namespace, topicsPattern, topics, - connection.ctx().executor(), topicListUpdateMaxQueueSize); - watcherRef.set(watcher); - topicResources.registerPersistentTopicListener(namespace, watcher); - // use updated permits to slow down responses so that backpressure gets applied - return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, - isPermitRequestCancelled, updatedPermits -> { - // reset retry backoff - retryBackoff.reset(); - // just return the watcher which was already created before - return CompletableFuture.completedFuture(watcher); - }, CompletableFuture::failedFuture); - }).whenComplete((watcher, exception) -> { - if (exception != null) { - TopicListWatcher w = watcherRef.get(); - if (w != null) { - w.close(); - topicResources.deregisterPersistentTopicListener(w); - } - // triggers a retry - throw FutureUtil.wrapToCompletionException(exception); - } else { - if (!watcherFuture.complete(watcher)) { - log.warn("[{}] Watcher future was already completed. Deregistering " - + "watcherId={}.", connection, watcherId); - watcher.close(); - topicResources.deregisterPersistentTopicListener(watcher); - watchers.remove(watcherId, watcherFuture); - } - } - }); - }, CompletableFuture::failedFuture); - }).exceptionally(t -> { - Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); - if (!isPermitRequestCancelled.getAsBoolean() && ( - unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException - || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { - // retry with backoff if permit acquisition fails due to timeout or queue full - long retryAfterMillis = this.retryBackoff.next(); - log.info("[{}] {} when initializing topic list watcher watcherId={} for namespace {}. Retrying in {} " - + "ms.", connection, unwrappedException.getMessage(), watcherId, namespace, - retryAfterMillis); - connection.ctx().executor() - .schedule(() -> initializeTopicsListWatcher(watcherFuture, namespace, watcherId, topicsPattern), - retryAfterMillis, TimeUnit.MILLISECONDS); + AtomicReference watcherRef = new AtomicReference<>(); + Consumer> afterListing = topics -> { + // register watcher immediately so that we don't lose events + TopicListWatcher watcher = + new TopicListWatcher(this, watcherId, namespace, topicsPattern, topics, + connection.ctx().executor(), topicListUpdateMaxQueueSize); + watcherRef.set(watcher); + topicResources.registerPersistentTopicListener(watcher); + }; + getTopics(namespace, watcherId, afterListing).whenComplete((topics, exception) -> { + TopicListWatcher w = watcherRef.get(); + if (exception != null) { + if (w != null) { + w.close(); + topicResources.deregisterPersistentTopicListener(w); + } + Throwable unwrappedException = FutureUtil.unwrapCompletionException(exception); + if (connection.isActive() && (unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException + || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { + // retry with backoff if permit acquisition fails due to timeout or queue full + long retryAfterMillis = this.retryBackoff.next(); + log.info("[{}] {} when initializing topic list watcher watcherId={} for namespace {}. " + + "Retrying in {} " + "ms.", connection, unwrappedException.getMessage(), watcherId, + namespace, retryAfterMillis); + connection.ctx().executor().schedule( + () -> initializeTopicsListWatcher(watcherFuture, namespace, watcherId, topicsPattern), + retryAfterMillis, TimeUnit.MILLISECONDS); + } else { + log.warn("[{}] Failed to initialize topic list watcher watcherId={} for namespace {}.", connection, + watcherId, namespace, unwrappedException); + watcherFuture.completeExceptionally(unwrappedException); + } } else { - log.warn("[{}] Failed to initialize topic list watcher watcherId={} for namespace {}.", connection, - watcherId, namespace, unwrappedException); - watcherFuture.completeExceptionally(unwrappedException); + if (!watcherFuture.complete(w)) { + log.warn("[{}] Watcher future was already completed. Deregistering " + "watcherId={}.", connection, + watcherId); + w.close(); + topicResources.deregisterPersistentTopicListener(w); + watchers.remove(watcherId, watcherFuture); + } } - return null; }); } - void updateTopicListWatcher(TopicListWatcher watcher) { - long watcherId = watcher.id; + private CompletableFuture> getTopics(NamespaceName namespace, long watcherId) { + return getTopics(namespace, watcherId, null); + } + + private CompletableFuture> getTopics(NamespaceName namespace, long watcherId, + Consumer> afterListing) { BooleanSupplier isPermitRequestCancelled = () -> !connection.isActive() || !watchers.containsKey(watcherId); if (isPermitRequestCancelled.getAsBoolean()) { - return; + return CompletableFuture.failedFuture( + new AsyncSemaphore.PermitAcquireCancelledException("Permit acquisition was cancelled")); } - NamespaceName namespace = watcher.namespace; + return getTopics(namespace, afterListing, isPermitRequestCancelled); + } + + private CompletableFuture> getTopics(NamespaceName namespace, + Consumer> afterListing, + BooleanSupplier isPermitRequestCancelled) { TopicListSizeResultCache.ResultHolder listSizeHolder = pulsar.getBrokerService().getTopicListSizeResultCache() .getTopicListSize(namespace.toString(), CommandGetTopicsOfNamespace.Mode.PERSISTENT); AsyncDualMemoryLimiter maxTopicListInFlightLimiter = pulsar.getBrokerService().getMaxTopicListInFlightLimiter(); - listSizeHolder.getSizeAsync().thenCompose(initialSize -> { + return listSizeHolder.getSizeAsync().thenCompose(initialSize -> { // use heap size limiter to avoid broker getting overwhelmed by a lot of concurrent topic list requests return maxTopicListInFlightLimiter.withAcquiredPermits(initialSize, - AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { - return namespaceService.getListOfPersistentTopics(namespace).thenCompose(topics -> { - long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); - listSizeHolder.updateSize(actualSize); - // use updated permits to slow down responses so that backpressure gets applied - return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, actualSize, - isPermitRequestCancelled, updatedPermits -> { + AsyncDualMemoryLimiter.LimitType.HEAP_MEMORY, isPermitRequestCancelled, initialPermits -> { + return namespaceService.getListOfUserTopics(namespace, + CommandGetTopicsOfNamespace.Mode.PERSISTENT).thenComposeAsync(topics -> { + long actualSize = TopicListMemoryLimiter.estimateTopicListSize(topics); + listSizeHolder.updateSize(actualSize); + if (afterListing != null) { + afterListing.accept(topics); + } + if (initialSize != actualSize) { + // use updated permits to slow down responses so that backpressure gets applied + return maxTopicListInFlightLimiter.withUpdatedPermits(initialPermits, + actualSize, + isPermitRequestCancelled, updatedPermits -> { + // reset retry backoff + retryBackoff.reset(); + // just return the topics which were already retrieved before + return CompletableFuture.completedFuture(topics); + }, CompletableFuture::failedFuture); + } else { // reset retry backoff retryBackoff.reset(); - // just return topics here return CompletableFuture.completedFuture(topics); - }, CompletableFuture::failedFuture); - }).whenComplete((topics, exception) -> { - if (exception != null) { - // triggers a retry - throw FutureUtil.wrapToCompletionException(exception); - } else { - watcher.updateTopics(topics); - } - }); - }, CompletableFuture::failedFuture); - }).exceptionally(t -> { - Throwable unwrappedException = FutureUtil.unwrapCompletionException(t); - if (!isPermitRequestCancelled.getAsBoolean() && ( - unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException - || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { - // retry with backoff if permit acquisition fails due to timeout or queue full - long retryAfterMillis = this.retryBackoff.next(); - log.info("[{}] {} when updating topic list watcher watcherId={} for namespace {}. Retrying in {} " - + "ms.", connection, unwrappedException.getMessage(), watcherId, namespace, - retryAfterMillis); - connection.ctx().executor() - .schedule(() -> updateTopicListWatcher(watcher), retryAfterMillis, TimeUnit.MILLISECONDS); + } + }, connection.ctx().executor()); + }, CompletableFuture::failedFuture) + .thenApplyAsync(Function.identity(), connection.ctx().executor()); + }); + } + + CompletableFuture updateTopicListWatcher(TopicListWatcher watcher) { + CompletableFuture future = new CompletableFuture<>(); + try { + internalUpdateTopicListWatcher(watcher, future); + } catch (Exception e) { + future.completeExceptionally(e); + } + return future; + } + + void internalUpdateTopicListWatcher(TopicListWatcher watcher, CompletableFuture future) { + NamespaceName namespace = watcher.namespace; + long watcherId = watcher.id; + getTopics(namespace, watcherId).whenComplete((topics, exception) -> { + if (exception != null) { + Throwable unwrappedException = FutureUtil.unwrapCompletionException(exception); + if (connection.isActive() && !watcher.closed + && (unwrappedException instanceof AsyncSemaphore.PermitAcquireTimeoutException + || unwrappedException instanceof AsyncSemaphore.PermitAcquireQueueFullException)) { + // retry with backoff if permit acquisition fails due to timeout or queue full + long retryAfterMillis = this.retryBackoff.next(); + log.info("[{}] {} when updating topic list watcher watcherId={} for namespace {}. Retrying in {} " + + "ms.", connection, unwrappedException.getMessage(), watcherId, namespace, + retryAfterMillis); + connection.ctx().executor() + .schedule(() -> internalUpdateTopicListWatcher(watcher, future), + retryAfterMillis, TimeUnit.MILLISECONDS); + } else { + log.warn("[{}] Failed to update topic list watcher watcherId={} for namespace {}.", connection, + watcherId, namespace, unwrappedException); + future.completeExceptionally(unwrappedException); + } } else { - log.warn("[{}] Failed to update topic list watcher watcherId={} for namespace {}.", connection, - watcherId, namespace, unwrappedException); + watcher.updateTopics(topics); + future.complete(null); } - return null; }); } 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 a19bc01757427..9044f4a910fb8 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 @@ -74,6 +74,7 @@ import java.util.stream.Collectors; import javax.naming.AuthenticationException; import lombok.AllArgsConstructor; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -235,7 +236,7 @@ public void setup() throws Exception { 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); + eq(NamespaceName.get("use", "ns-abc")), any()); doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfPersistentTopics( NamespaceName.get("use", "ns-abc")); doReturn(CompletableFuture.completedFuture(TopicExistsInfo.newTopicNotExists())).when(namespaceService) @@ -3253,6 +3254,8 @@ public void testGetTopicsOfNamespaceNoChange() throws Exception { public void testWatchTopicList() throws Exception { svcConfig.setEnableBrokerSideSubscriptionPatternEvaluation(true); resetChannel(); + @Cleanup + BackGroundExecutor backGroundExecutor = startBackgroundExecutorForEmbeddedChannel(channel); setChannelConnected(); BaseCommand command = Commands.newWatchTopicList(1, 3, "use/ns-abc", "use/ns-abc/topic-.*", null); ByteBuf serializedCommand = Commands.serializeWithSize(command); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index d85f244b9a373..5caa771289114 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -21,8 +21,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; @@ -43,6 +45,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -62,17 +65,20 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.topiclistlimit.TopicListSizeResultCache; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; 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.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.semaphore.AsyncSemaphore; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.topics.TopicsPattern; -import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.awaitility.Awaitility; import org.jspecify.annotations.NonNull; import org.mockito.InOrder; @@ -97,6 +103,7 @@ public class TopicListServiceTest { private AsyncDualMemoryLimiterImpl memoryLimiter; private ScheduledExecutorService scheduledExecutorService; private PulsarService pulsar; + private NamespaceService namespaceService; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { @@ -105,7 +112,7 @@ public void setup() throws Exception { topicListFuture = new CompletableFuture<>(); AtomicReference> listenerRef = new AtomicReference<>(); - MetadataStore metadataStore = mock(MetadataStore.class); + MetadataStoreExtended metadataStore = mock(MetadataStoreExtended.class); doAnswer(invocationOnMock -> { listenerRef.set(invocationOnMock.getArgument(0)); return null; @@ -114,10 +121,10 @@ public void setup() throws Exception { notificationConsumer = listenerRef.get(); pulsar = mock(PulsarService.class); - NamespaceService namespaceService = mock(NamespaceService.class); + namespaceService = mock(NamespaceService.class); when(pulsar.getNamespaceService()).thenReturn(namespaceService); doAnswer(invocationOnMock -> topicListFuture) - .when(namespaceService).getListOfPersistentTopics(any()); + .when(namespaceService).getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); when(pulsar.getPulsarResources()).thenReturn(mock(PulsarResources.class)); when(pulsar.getPulsarResources().getTopicResources()).thenReturn(topicResources); @@ -200,8 +207,7 @@ public void testCommandWatchSuccessResponse() { String hash = TopicList.calculateHash(topics); topicListFuture.complete(topics); Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); - verify(topicResources).registerPersistentTopicListener( - eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); + verify(topicResources).registerPersistentTopicListener(any(TopicListService.TopicListWatcher.class)); Collection expectedTopics = new ArrayList<>(topics); verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(expectedTopics), any()); @@ -229,8 +235,7 @@ public void testCommandWatchSuccessResponseWhenOutOfPermits() throws ExecutionEx // release the permits memoryLimiter.release(permit); Awaitility.await().untilAsserted(() -> Assert.assertEquals(1, lookupSemaphore.availablePermits())); - verify(topicResources).registerPersistentTopicListener( - eq(NamespaceName.get("tenant/ns")), any(TopicListService.TopicListWatcher.class)); + verify(topicResources).registerPersistentTopicListener(any(TopicListService.TopicListWatcher.class)); Collection expectedTopics = new ArrayList<>(topics); verify(connection.getCommandSender()).sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(expectedTopics), any()); @@ -382,19 +387,15 @@ public void testCommandWatchUpdateQueueOverflows() { topicListFuture.complete(topics); assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); - CompletableFuture completePending = new CompletableFuture<>(); - doReturn(completePending).when(pulsarCommandSender) - .sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); topicListFuture = new CompletableFuture<>(); - + List updatedTopics = IntStream.range(1, 101).mapToObj(i -> "persistent://tenant/ns/topic" + i).toList(); // when the queue overflows - for (int i = 10; i <= 10 + topicListUpdateMaxQueueSize + 1; i++) { - notificationConsumer.accept( - new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic" + i)); + for (int i = 1; i < updatedTopics.size(); i++) { + TopicName topicName = TopicName.get(updatedTopics.get(i)); + notificationConsumer.accept(new Notification(NotificationType.Created, + "/managed-ledgers/" + topicName.getPersistenceNamingEncoding())); } - - // a new listing should be performed. Return 100 topics in the response, simulating that events have been lost - List updatedTopics = IntStream.range(1, 101).mapToObj(i -> "persistent://tenant/ns/topic" + i).toList(); + // a new listing should be performed topicListFuture.complete(updatedTopics); // validate that the watcher's matching topics have been updated Awaitility.await().untilAsserted(() -> { @@ -405,4 +406,330 @@ public void testCommandWatchUpdateQueueOverflows() { }); } + @Test + public void testCommandWatchSuccessNoTopicsInResponseWhenHashMatches() { + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + String hash = TopicList.calculateHash(topics); + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, hash, + lookupSemaphore); + doReturn(CompletableFuture.completedFuture(null)).when(pulsarCommandSender) + .sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any()); + topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + Collection expectedTopics = List.of(); + verify(connection.getCommandSender(), timeout(2000L).times(1)) + .sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(expectedTopics), any()); + } + + @Test + public void testCommandWatchSuccessTopicsInResponseWhenHashDoesntMatch() { + List topics = Collections.singletonList("persistent://tenant/ns/topic1"); + String hash = TopicList.calculateHash(topics); + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, "INVALID_HASH", + lookupSemaphore); + doReturn(CompletableFuture.completedFuture(null)).when(pulsarCommandSender) + .sendWatchTopicListSuccess(anyLong(), anyLong(), anyString(), any(), any()); + topicListFuture.complete(topics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + Collection expectedTopics = topics; + verify(connection.getCommandSender(), timeout(2000L).times(1)) + .sendWatchTopicListSuccess(eq(7L), eq(13L), eq(hash), eq(expectedTopics), any()); + } + + @Test + public void testSessionDisconnectAndReconnectSendsNewAndDeletedTopics() { + // Initial topics: topic1, topic2, topic3 + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2", + "persistent://tenant/ns/topic3"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(initialTopics); + + // Simulate session disconnect + watcher.onSessionEvent(SessionEvent.ConnectionLost); + + // Prepare topics after reconnect: topic2 remains, topic1 and topic3 deleted, topic4 and topic5 added + List topicsAfterReconnect = List.of( + "persistent://tenant/ns/topic2", + "persistent://tenant/ns/topic4", + "persistent://tenant/ns/topic5"); + + topicListFuture = new CompletableFuture<>(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocationOnMock -> topicListFuture) + .when(namespaceService).getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + // Simulate session reconnect - this should trigger a topic list refresh + watcher.onSessionEvent(SessionEvent.Reconnected); + + // Complete the topic list future with the new topics + topicListFuture.complete(topicsAfterReconnect); + + // Expected: deleted topics are topic1 and topic3, new topics are topic4 and topic5 + List expectedDeleted = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic3"); + List expectedNew = List.of( + "persistent://tenant/ns/topic4", + "persistent://tenant/ns/topic5"); + String expectedHash = TopicList.calculateHash(topicsAfterReconnect); + + // Verify that sendWatchTopicListUpdate is called with the correct new and deleted topics + verify(connection.getCommandSender(), timeout(2000L)) + .sendWatchTopicListUpdate(eq(13L), + argThat(newTopics -> new HashSet<>(newTopics).equals(new HashSet<>(expectedNew))), + argThat(deletedTopics -> new HashSet<>(deletedTopics).equals(new HashSet<>(expectedDeleted))), + eq(expectedHash), any()); + + // Verify the watcher's matching topics are updated + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(topicsAfterReconnect); + } + + @Test + public void testSessionLostAndReestablishedSendsNewAndDeletedTopics() { + // Initial topics: topic1, topic2 + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + + // Simulate session lost (more severe than connection lost) + watcher.onSessionEvent(SessionEvent.SessionLost); + + // After reconnect: all topics deleted, completely new topics added + List topicsAfterReconnect = List.of( + "persistent://tenant/ns/topic7", + "persistent://tenant/ns/topic8", + "persistent://tenant/ns/topic9"); + + topicListFuture = new CompletableFuture<>(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocationOnMock -> topicListFuture) + .when(namespaceService).getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + // Simulate session reestablished + watcher.onSessionEvent(SessionEvent.SessionReestablished); + + topicListFuture.complete(topicsAfterReconnect); + + List expectedDeleted = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + List expectedNew = List.of( + "persistent://tenant/ns/topic7", + "persistent://tenant/ns/topic8", + "persistent://tenant/ns/topic9"); + String expectedHash = TopicList.calculateHash(topicsAfterReconnect); + + verify(connection.getCommandSender(), timeout(2000L)) + .sendWatchTopicListUpdate(eq(13L), + argThat(newTopics -> new HashSet<>(newTopics).equals(new HashSet<>(expectedNew))), + argThat(deletedTopics -> new HashSet<>(deletedTopics).equals(new HashSet<>(expectedDeleted))), + eq(expectedHash), any()); + + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(topicsAfterReconnect); + } + + @Test + public void testSessionReconnectWithNoChangesDoesNotSendUpdate() { + // Initial topics + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + + // Simulate session disconnect + watcher.onSessionEvent(SessionEvent.ConnectionLost); + + // Topics remain the same after reconnect + topicListFuture = new CompletableFuture<>(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocationOnMock -> topicListFuture) + .when(namespaceService).getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + watcher.onSessionEvent(SessionEvent.Reconnected); + + // Complete with the same topics + topicListFuture.complete(initialTopics); + + // Wait for processing + Awaitility.await().untilAsserted(() -> + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(initialTopics)); + + // Verify that sendWatchTopicListUpdate was NOT called (no changes to report) + verify(connection.getCommandSender(), timeout(500L).times(0)) + .sendWatchTopicListUpdate(eq(13L), any(), any(), any(), any()); + } + + @Test + public void testConnectionOrSessionFlappingDoesNotTriggerOverlappingUpdate() { + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + clearInvocations(namespaceService); + + topicListFuture = new CompletableFuture<>(); + watcher.onSessionEvent(SessionEvent.ConnectionLost); + watcher.onSessionEvent(SessionEvent.Reconnected); + + Awaitility.await().untilAsserted(() -> verify(namespaceService, timeout(500L).times(1)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT))); + + watcher.onSessionEvent(SessionEvent.ConnectionLost); + watcher.onSessionEvent(SessionEvent.Reconnected); + watcher.onSessionEvent(SessionEvent.ConnectionLost); + watcher.onSessionEvent(SessionEvent.Reconnected); + watcher.onSessionEvent(SessionEvent.SessionLost); + watcher.onSessionEvent(SessionEvent.SessionReestablished); + + verify(namespaceService, timeout(500L).times(1)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + topicListFuture.complete(initialTopics); + Awaitility.await().untilAsserted(() -> + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(initialTopics)); + } + + @Test + public void testWatchTopicListReconcileDoesntTriggerOverlappingUpdate() { + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + + clearInvocations(namespaceService); + topicListFuture = new CompletableFuture<>(); + watcher.onSessionEvent(SessionEvent.ConnectionLost); + watcher.onSessionEvent(SessionEvent.Reconnected); + Awaitility.await().untilAsserted(() -> verify(namespaceService, timeout(500L).times(1)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT))); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, TopicList.calculateHash(initialTopics), + lookupSemaphore); + + topicListFuture.complete(initialTopics); + + verify(namespaceService, timeout(500L).times(1)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + Awaitility.await().untilAsserted(() -> + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(initialTopics)); + } + + @Test + public void testWatchTopicListReconcileNoOverlappingUpdate() { + List initialTopics = List.of( + "persistent://tenant/ns/topic1", + "persistent://tenant/ns/topic2"); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, null, + lookupSemaphore); + topicListFuture.complete(initialTopics); + assertThat(topicListService.getWatcherFuture(13)).succeedsWithin(Duration.ofSeconds(2)); + + TopicListService.TopicListWatcher watcher = topicListService.getWatcherFuture(13).join(); + + clearInvocations(namespaceService); + topicListFuture = new CompletableFuture<>(); + watcher.onSessionEvent(SessionEvent.ConnectionLost); + watcher.onSessionEvent(SessionEvent.Reconnected); + Awaitility.await().untilAsserted(() -> verify(namespaceService, timeout(500L).times(1)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT))); + + topicListFuture.complete(initialTopics); + Awaitility.await().untilAsserted(() -> { + assertThat(watcher.isUpdatingTopics()).isFalse(); + }); + + topicListService.handleWatchTopicList( + NamespaceName.get("tenant/ns"), + 13, + 7, + "persistent://tenant/ns/topic\\d", + topicsPatternImplementation, TopicList.calculateHash(initialTopics), + lookupSemaphore); + + verify(namespaceService, timeout(1000L).times(2)) + .getListOfUserTopics(any(), eq(CommandGetTopicsOfNamespace.Mode.PERSISTENT)); + + Awaitility.await().untilAsserted(() -> + assertThat(watcher.getMatchingTopics()).containsExactlyInAnyOrderElementsOf(initialTopics)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java index 46262e84d388b..1a75dd70baec9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListWatcherTest.java @@ -75,7 +75,7 @@ public void testGetMatchingTopicsReturnsFilteredList() { @Test public void testAcceptSendsNotificationAndRemembersTopic() { String newTopic = "persistent://tenant/ns/topic3"; - watcher.accept(newTopic, NotificationType.Created); + watcher.onTopicEvent(newTopic, NotificationType.Created); List allMatchingTopics = Arrays.asList( "persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2", newTopic); @@ -90,7 +90,7 @@ public void testAcceptSendsNotificationAndRemembersTopic() { @Test public void testAcceptSendsNotificationAndForgetsTopic() { String deletedTopic = "persistent://tenant/ns/topic1"; - watcher.accept(deletedTopic, NotificationType.Deleted); + watcher.onTopicEvent(deletedTopic, NotificationType.Deleted); List allMatchingTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(allMatchingTopics); @@ -103,7 +103,7 @@ public void testAcceptSendsNotificationAndForgetsTopic() { @Test public void testAcceptIgnoresNonMatching() { - watcher.accept("persistent://tenant/ns/mytopic", NotificationType.Created); + watcher.onTopicEvent("persistent://tenant/ns/mytopic", NotificationType.Created); verifyNoInteractions(topicListService); Assert.assertEquals( Arrays.asList("persistent://tenant/ns/topic1", "persistent://tenant/ns/topic2"), @@ -114,7 +114,7 @@ public void testAcceptIgnoresNonMatching() { public void testUpdateQueueOverFlowPerformsFullUpdate() { for (int i = 10; i <= 20; i++) { String newTopic = "persistent://tenant/ns/topic" + i; - watcher.accept(newTopic, NotificationType.Created); + watcher.onTopicEvent(newTopic, NotificationType.Created); } verify(topicListService).sendTopicListUpdate(anyLong(), anyString(), any(), any(), any()); verify(topicListService).updateTopicListWatcher(any()); 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 46e41be012511..7fba9c9fc28af 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 @@ -149,7 +149,7 @@ static class TestPulsarResources extends PulsarResources { private final TopicResources topicResources; private final NamespaceResources namespaceResources; - public TestPulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, + public TestPulsarResources(MetadataStoreExtended localMetadataStore, MetadataStore configurationMetadataStore, TopicResources topicResources, NamespaceResources namespaceResources) { super(localMetadataStore, configurationMetadataStore); this.topicResources = topicResources; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerTopicWatcherBackPressureMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerTopicWatcherBackPressureMultipleConsumersTest.java index e47c85d16d949..8fcbb89a931a2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerTopicWatcherBackPressureMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternConsumerTopicWatcherBackPressureMultipleConsumersTest.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -32,13 +31,13 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.TopicListWatcher; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiter; import org.apache.pulsar.common.semaphore.AsyncDualMemoryLimiterImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -136,15 +135,11 @@ public void testPatternConsumerWithLargeAmountOfConcurrentClientConnections() List> consumers = consumerFutures.stream().map(CompletableFuture::join).toList(); - List> watcherFutures = consumers.stream().map(consumer -> { - try { - CompletableFuture watcherFuture = consumer instanceof PatternMultiTopicsConsumerImpl - ? (CompletableFuture) FieldUtils.readField(consumer, "watcherFuture", true) : null; - return watcherFuture; - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } - }).filter(Objects::nonNull).toList(); + List> watcherFutures = + consumers.stream().filter(PatternMultiTopicsConsumerImpl.class::isInstance) + .map(PatternMultiTopicsConsumerImpl.class::cast) + .map(c -> (CompletableFuture) c.getWatcherFuture()) + .toList(); // wait for all watcher futures to complete FutureUtil.waitForAll(watcherFutures).get(60, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConSupports.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConSupports.java index b8855656ad026..63e4d513a6900 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConSupports.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConSupports.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import static org.assertj.core.api.Assertions.assertThat; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -191,4 +192,12 @@ protected void ensureTransactionWorks(PulsarClientImpl pulsarClient, Producer pr Assert.assertEquals(new String(messageTx.getData(), StandardCharsets.UTF_8), messageContentTx); consumer.acknowledge(messageTx); } + + protected void waitForTopicListWatcherStarted(Consumer consumer) { + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = + ((PatternMultiTopicsConsumerImpl) consumer).getWatcherFuture(); + assertThat(completableFuture).describedAs("Topic list watcher future should be done").isDone(); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTopicsPatternConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTopicsPatternConsumerTest.java index 981612d3930fd..be4ca3c4a50df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTopicsPatternConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConTopicsPatternConsumerTest.java @@ -72,6 +72,9 @@ public void testConnectionAutoReleaseWhileUsingTopicsPatternConsumer() throws Ex .subscriptionName("my-subscription-y") .subscribe(); + waitForTopicListWatcherStarted(consumer); + waitForTopicListWatcherStarted(consumer2); + // check that there are more than 3 connections // at least 3 connections are required: // 1 for "producer", 1 for "consumer", and 1 for the topic watcher of "consumer" 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 c1b96a058f4ad..ab4ed0dd95e33 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 @@ -18,9 +18,9 @@ */ package org.apache.pulsar.client.impl; -import static org.mockito.Mockito.doReturn; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -35,7 +35,6 @@ import java.util.regex.Pattern; import java.util.stream.IntStream; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.InjectedClientCnxClientBuilder; import org.apache.pulsar.client.api.Message; @@ -55,7 +54,6 @@ import org.apache.pulsar.common.naming.TopicName; 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; @@ -200,11 +198,11 @@ public void testBinaryProtoToGetTopicsOfNamespacePersistent() throws Exception { .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer4 = pulsarClient.newProducer().topic(topicName4) .enableBatching(false) @@ -221,10 +219,7 @@ public void testBinaryProtoToGetTopicsOfNamespacePersistent() throws Exception { 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()); - }); + waitTopicListWatcherCreation(consumer); // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); @@ -305,17 +300,18 @@ public void testBinaryProtoSubscribeAllTopicOfNamespace() throws Exception { 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()); - }); + waitTopicListWatcherCreation(consumer); // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); - assertEquals(topics.size(), 6); + List expectedTopics = + List.of(topicName1 + "-partition-0", + topicName2 + "-partition-0", topicName2 + "-partition-1", + topicName3 + "-partition-0", topicName3 + "-partition-1", topicName3 + "-partition-2"); + assertThat(topics).containsExactlyInAnyOrderElementsOf(expectedTopics); assertEquals(consumers.size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 3); @@ -368,11 +364,11 @@ public void testBinaryProtoToGetTopicsOfNamespaceNonPersistent() throws Exceptio .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer4 = pulsarClient.newProducer().topic(topicName4) .enableBatching(false) @@ -388,10 +384,7 @@ public void testBinaryProtoToGetTopicsOfNamespaceNonPersistent() throws Exceptio .subscribe(); // Wait topic list watcher creation. - Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); - assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); - }); + waitTopicListWatcherCreation(consumer); // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); @@ -466,11 +459,11 @@ public void testBinaryProtoToGetTopicsOfNamespaceAll() throws Exception { .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer4 = pulsarClient.newProducer().topic(topicName4) .enableBatching(false) @@ -486,10 +479,7 @@ public void testBinaryProtoToGetTopicsOfNamespaceAll() throws Exception { .subscribe(); // Wait topic list watcher creation. - Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); - assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); - }); + waitTopicListWatcherCreation(consumer); // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); @@ -563,10 +553,7 @@ public void testStartEmptyPatternConsumer() throws Exception { .receiverQueueSize(4) .subscribe(); // Wait topic list watcher creation. - Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); - assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); - }); + waitTopicListWatcherCreation(consumer); // 3. verify consumer get methods, to get 5 number of partitions and topics. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); @@ -584,11 +571,11 @@ public void testStartEmptyPatternConsumer() throws Exception { .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); // 5. verify consumer get methods, to get number of partitions and topics, value 6=1+2+3. @@ -694,10 +681,7 @@ public void testSubscribePatterWithOutTopicDomain() throws Exception { .subscribe(); // 0. Need make sure topic watcher started - waitForTopicListWatcherStarted(consumer); - - // if broker enable watch topic, then recheckPatternTimeout will be null. - assertNull(((PatternMultiTopicsConsumerImpl) consumer).getRecheckPatternTimeout()); + waitTopicListWatcherCreation(consumer); // 1. create partition topic String topicName = "persistent://my-property/my-ns/test-pattern" + key; @@ -717,6 +701,59 @@ public void testSubscribePatterWithOutTopicDomain() throws Exception { pulsarClient.close(); } + @Test(timeOut = testTimeout) + public void testPeriodicReconciliationWithActiveWatcher() throws Exception { + String key = "PeriodicReconciliationWithActiveWatcher"; + String subscriptionName = "my-ex-subscription-" + key; + String topicName = "persistent://my-property/my-ns/pattern-topic-1-" + key; + Pattern pattern = Pattern.compile("persistent://my-property/my-ns/pattern-topic.*"); + + // Create a topic so consumer has something to subscribe to + admin.topics().createPartitionedTopic(topicName, 2); + + // Create consumer with short auto-discovery period (1 second) + Consumer consumer = pulsarClient.newConsumer() + .topicsPattern(pattern) + .patternAutoDiscoveryPeriod(1, TimeUnit.SECONDS) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + try { + // Wait for topic list watcher to be connected + waitTopicListWatcherCreation(consumer); + + PatternMultiTopicsConsumerImpl patternConsumer = (PatternMultiTopicsConsumerImpl) consumer; + + // Verify that recheckPatternTimeout is NOT null even with active watcher + // This confirms the periodic timer is always scheduled + assertNotNull(patternConsumer.getRecheckPatternTimeout(), + "recheckPatternTimeout should not be null even when TopicListWatcher is connected"); + + // Record the initial recheck epoch + int initialEpoch = patternConsumer.getRecheckPatternEpoch(); + + // Wait for at least 2 reconciliation cycles (with 1 second period) + Awaitility.await() + .atMost(Duration.ofSeconds(10)) + .pollInterval(Duration.ofMillis(500)) + .untilAsserted(() -> { + int currentEpoch = patternConsumer.getRecheckPatternEpoch(); + assertTrue(currentEpoch > initialEpoch + 1, + "recheckPatternEpoch should increase over time, indicating periodic reconciliation. " + + "Initial: " + initialEpoch + ", Current: " + currentEpoch); + }); + + // Verify timeout is still scheduled after reconciliation cycles + assertNotNull(patternConsumer.getRecheckPatternTimeout(), + "recheckPatternTimeout should remain scheduled after reconciliation cycles"); + + } finally { + consumer.close(); + admin.topics().deletePartitionedTopic(topicName); + } + } + @DataProvider(name = "topicDomain") public Object[][] topicDomain(){ return new Object[][]{ @@ -788,9 +825,10 @@ public Object[][] regexpConsumerArgs(){ }; } - private void waitForTopicListWatcherStarted(Consumer consumer) { + private void waitTopicListWatcherCreation(Consumer consumer) { Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + CompletableFuture completableFuture = + ((PatternMultiTopicsConsumerImpl) consumer).getWatcherFuture(); log.info("isDone: {}, isCompletedExceptionally: {}", completableFuture.isDone(), completableFuture.isCompletedExceptionally()); assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); @@ -814,7 +852,7 @@ public void testPreciseRegexpSubscribe(boolean partitioned, boolean createTopicA .receiverQueueSize(4) .subscribe(); if (createTopicAfterWatcherStarted) { - waitForTopicListWatcherStarted(consumer); + waitTopicListWatcherCreation(consumer); } // 1. create topic. @@ -867,7 +905,7 @@ public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) .topicsPattern(pattern) // Disable brokerSideSubscriptionPatternEvaluation will leading disable topic list watcher. // So set patternAutoDiscoveryPeriod to a little value. - .patternAutoDiscoveryPeriod(1) + .patternAutoDiscoveryPeriod(5, TimeUnit.SECONDS) .subscriptionName(subscriptionName) .subscriptionType(SubscriptionType.Shared) .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) @@ -882,9 +920,9 @@ public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) } // 2. verify consumer can subscribe the topic. - // Since the minimum value of `patternAutoDiscoveryPeriod` is 60s, we set the test timeout to a triple value. + // Since the minimum value of `patternAutoDiscoveryPeriod` is 5s, we set the test timeout to a triple value. assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().inputPattern()); - Awaitility.await().atMost(Duration.ofMinutes(3)).untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 1); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 1); if (partitioned) { @@ -955,11 +993,11 @@ public void testAutoSubscribePatternConsumer() throws Exception { .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Consumer consumer = pulsarClient.newConsumer() @@ -972,10 +1010,7 @@ public void testAutoSubscribePatternConsumer() throws Exception { .subscribe(); // Wait topic list watcher creation. - Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); - assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); - }); + waitTopicListWatcherCreation(consumer); assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); @@ -1008,7 +1043,7 @@ public void testAutoSubscribePatternConsumer() throws Exception { admin.topics().createPartitionedTopic(topicName4, 4); Producer producer4 = pulsarClient.newProducer().topic(topicName4) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); // 7. call recheckTopics to subscribe each added topics above, verify topics number: 10=1+2+3+4 @@ -1070,27 +1105,24 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { .create(); Producer producer2 = pulsarClient.newProducer().topic(topicName2) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Producer producer3 = pulsarClient.newProducer().topic(topicName3) .enableBatching(false) - .messageRoutingMode(org.apache.pulsar.client.api.MessageRoutingMode.RoundRobinPartition) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) .create(); Consumer consumer = pulsarClient.newConsumer() - .topicsPattern(pattern) - .patternAutoDiscoveryPeriod(10, TimeUnit.SECONDS) - .subscriptionName(subscriptionName) - .subscriptionType(SubscriptionType.Shared) - .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) - .receiverQueueSize(4) - .subscribe(); + .topicsPattern(pattern) + .patternAutoDiscoveryPeriod(3, TimeUnit.SECONDS) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Shared) + .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()); - }); + waitTopicListWatcherCreation(consumer); assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); @@ -1118,23 +1150,27 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { } while (message != null); assertEquals(messageSet, totalMessages); - // 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. - 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")); + // 6. remove producer 1,3 and delete topics; verify only consumer 2 left + producer3.close(); + producer1.close(); // 7. call recheckTopics to unsubscribe topic 1,3, verify topics number: 2=6-1-3 log.debug("recheck topics change"); - PatternConsumerUpdateQueue taskQueue = WhiteboxImpl.getInternalState(consumer, "updateTaskQueue"); - taskQueue.appendRecheckOp(); - Thread.sleep(100); - assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 2); - assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 2); - assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 1); + Awaitility.await().untilAsserted(() -> { + try { + admin.topics().deletePartitionedTopic(topicName3, true); + } catch (Exception e) { + // ignore + } + try { + admin.topics().delete(topicName1, true); + } catch (Exception e) { + // ignore + } + assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 2); + assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 2); + assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 1); + }); // 8. produce data to topic2, verify should receive all the message for (int i = 0; i < totalMessages; i++) { @@ -1179,10 +1215,7 @@ public void testTopicDeletion() throws Exception { .subscribe(); // Wait topic list watcher creation. - Awaitility.await().untilAsserted(() -> { - CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); - assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); - }); + waitTopicListWatcherCreation(consumer); assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); PatternMultiTopicsConsumerImpl consumerImpl = (PatternMultiTopicsConsumerImpl) consumer; 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 041f944feb479..91b44b4fb10b7 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 @@ -189,9 +189,14 @@ public class ClientCnx extends PulsarHandler { @Getter protected AuthenticationDataProvider authenticationDataProvider; private TransactionBufferHandler transactionBufferHandler; + @Getter private boolean supportsTopicWatchers; @Getter private boolean supportsGetPartitionedMetadataWithoutAutoCreation; + @Getter + private boolean brokerSupportsReplDedupByLidAndEid; + @Getter + private boolean supportsTopicWatcherReconcile; /** Idle stat. **/ @Getter @@ -405,6 +410,10 @@ protected void handleConnected(CommandConnected connected) { supportsGetPartitionedMetadataWithoutAutoCreation = connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsGetPartitionedMetadataWithoutAutoCreation(); + brokerSupportsReplDedupByLidAndEid = + connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsReplDedupByLidAndEid(); + supportsTopicWatcherReconcile = + connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsTopicWatcherReconcile(); // set remote protocol version to the correct version before we complete the connection future setRemoteEndpointProtocolVersion(connected.getProtocolVersion()); @@ -1215,6 +1224,16 @@ public CompletableFuture newWatchTopicList( RequestType.Command, true); } + /** + * Create and send a WatchTopicList request including the topics-hash. + * Delegates to the existing BaseCommand-based method after building the command. + */ + public CompletableFuture newWatchTopicList( + long requestId, long watcherId, String namespace, String topicsPattern, String topicsHash) { + BaseCommand cmd = Commands.newWatchTopicList(requestId, watcherId, namespace, topicsPattern, topicsHash); + return newWatchTopicList(cmd, requestId); + } + public CompletableFuture newWatchTopicListClose( BaseCommand commandWatchTopicListClose, long requestId) { return sendRequestAndHandleTimeout( 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 index d6eba6463a07d..49b0add21e7b5 100644 --- 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 @@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; /** * Used to make all tasks that will modify subscriptions will be executed one by one, and skip the unnecessary updating. @@ -47,15 +48,76 @@ @Slf4j @SuppressFBWarnings("EI_EXPOSE_REP2") public class PatternConsumerUpdateQueue { + private final LinkedBlockingQueue pendingTasks; - private static final Pair> RECHECK_OP = - Pair.of(UpdateSubscriptionType.RECHECK, null); + private final PatternMultiTopicsConsumerImpl patternConsumer; - private final LinkedBlockingQueue>> pendingTasks; + private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; - private final PatternMultiTopicsConsumerImpl patternConsumer; + static class UpdateTask { + private final UpdateSubscriptionType type; - private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + UpdateTask(UpdateSubscriptionType type) { + this.type = type; + } + } + + static class RecheckTask extends UpdateTask { + public static final RecheckTask INSTANCE = new RecheckTask(); + private RecheckTask() { + super(UpdateSubscriptionType.RECHECK); + } + + @Override + public String toString() { + return "RecheckTask"; + } + } + + static class InitTask extends UpdateTask { + public static final InitTask INSTANCE = new InitTask(); + private InitTask() { + super(UpdateSubscriptionType.CONSUMER_INIT); + } + + @Override + public String toString() { + return "InitTask"; + } + } + + static class TopicsAddedOrRemovedTask extends UpdateTask { + private final Collection addedTopics; + private final Collection removedTopics; + private final String topicsHash; + + public TopicsAddedOrRemovedTask(Collection addedTopics, Collection removedTopics, + String topicsHash) { + super(UpdateSubscriptionType.TOPICS_CHANGED); + this.addedTopics = addedTopics; + this.removedTopics = removedTopics; + this.topicsHash = topicsHash; + } + + @Override + public String toString() { + return "TopicsAddedOrRemovedTask{" + "addedTopics=" + addedTopics + ", removedTopics=" + removedTopics + + ", topicsHash='" + topicsHash + '\'' + '}'; + } + } + + static class WatchTopicListSuccessTask extends UpdateTask { + private final CommandWatchTopicListSuccess response; + private final String localStateTopicsHash; + private final int epoch; + + WatchTopicListSuccessTask(CommandWatchTopicListSuccess response, String localStateTopicsHash, int epoch) { + super(UpdateSubscriptionType.WATCH_TOPIC_LIST_SUCCESS); + this.response = response; + this.localStateTopicsHash = localStateTopicsHash; + this.epoch = epoch; + } + } /** * Whether there is a task is in progress, this variable is used to confirm whether a next-task triggering is @@ -75,44 +137,40 @@ public class PatternConsumerUpdateQueue { private boolean closed; - public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer) { + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer) { this(patternConsumer, patternConsumer.topicsChangeListener); } /** This constructor is only for test. **/ @VisibleForTesting - public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, + @SuppressWarnings("this-escape") + 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)); + doAppend(InitTask.INSTANCE); } - synchronized void appendTopicsAddedOp(Collection topics) { - if (topics == null || topics.isEmpty()) { - return; - } - doAppend(Pair.of(UpdateSubscriptionType.TOPICS_ADDED, topics)); + synchronized void appendTopicsChangedOp(Collection addedTopics, Collection deletedTopics, + String topicsHash) { + doAppend(new TopicsAddedOrRemovedTask(addedTopics, deletedTopics, topicsHash)); } - synchronized void appendTopicsRemovedOp(Collection topics) { - if (topics == null || topics.isEmpty()) { - return; - } - doAppend(Pair.of(UpdateSubscriptionType.TOPICS_REMOVED, topics)); + synchronized void appendRecheckOp() { + doAppend(RecheckTask.INSTANCE); } - synchronized void appendRecheckOp() { - doAppend(RECHECK_OP); + synchronized void appendWatchTopicListSuccessOp(CommandWatchTopicListSuccess response, String localStateTopicsHash, + int epoch) { + doAppend(new WatchTopicListSuccessTask(response, localStateTopicsHash, epoch)); } - synchronized void doAppend(Pair> task) { + synchronized void doAppend(UpdateTask task) { if (log.isDebugEnabled()) { - log.debug("Pattern consumer [{}] try to append task. {} {}", patternConsumer.getSubscription(), - task.getLeft(), task.getRight() == null ? "" : task.getRight()); + log.debug("Pattern consumer [{}] try to append task. {}", patternConsumer.getSubscription(), task); } // Once there is a recheck task in queue, it means other tasks can be skipped. if (recheckTaskInQueue) { @@ -120,13 +178,13 @@ synchronized void doAppend(Pair> task } // Once there are too many tasks in queue, compress them as a recheck task. - if (pendingTasks.size() >= 30 && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + if (pendingTasks.size() >= 30 && task.type != UpdateSubscriptionType.RECHECK) { appendRecheckOp(); return; } pendingTasks.add(task); - if (task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + if (task.type == UpdateSubscriptionType.RECHECK) { recheckTaskInQueue = true; } @@ -141,7 +199,7 @@ synchronized void triggerNextTask() { return; } - final Pair> task = pendingTasks.poll(); + final UpdateTask task = pendingTasks.poll(); // No pending task. if (task == null) { @@ -150,14 +208,14 @@ synchronized void triggerNextTask() { } // If there is a recheck task in queue, skip others and only call the recheck task. - if (recheckTaskInQueue && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + if (recheckTaskInQueue && task.type != UpdateSubscriptionType.RECHECK) { triggerNextTask(); return; } // Execute pending task. CompletableFuture newTaskFuture = null; - switch (task.getLeft()) { + switch (task.type) { case CONSUMER_INIT: { newTaskFuture = patternConsumer.getSubscribeFuture().thenAccept(__ -> {}).exceptionally(ex -> { // If the subscribe future was failed, the consumer will be closed. @@ -173,12 +231,32 @@ synchronized void triggerNextTask() { }); break; } - case TOPICS_ADDED: { - newTaskFuture = topicsChangeListener.onTopicsAdded(task.getRight()); - break; - } - case TOPICS_REMOVED: { - newTaskFuture = topicsChangeListener.onTopicsRemoved(task.getRight()); + case TOPICS_CHANGED: { + TopicsAddedOrRemovedTask topicsAddedOrRemovedTask = (TopicsAddedOrRemovedTask) task; + newTaskFuture = topicsChangeListener.onTopicsRemoved(topicsAddedOrRemovedTask.removedTopics) + .thenCompose(__ -> + topicsChangeListener.onTopicsAdded(topicsAddedOrRemovedTask.addedTopics)) + .thenRun(() -> { + if (!patternConsumer.supportsTopicListWatcherReconcile()) { + // Ignore the topics hash until topic-list watcher reconciliation is supported. + // Broker-side state can be stale, which would trigger unnecessary reconciliation. + // The client will reconcile later when it fetches the topic list after the next + // patternAutoDiscoveryPeriod interval. + // Brokers that support watcher reconciliation also refresh broker-side state + // when reconciliation is requested. + // Older brokers have known topic-listing bugs (issue 25192: system topics included), + // so their hash is not reliable anyway. + return; + } + String localHash = patternConsumer.getLocalStateTopicsHash(); + String brokerHash = topicsAddedOrRemovedTask.topicsHash; + if (brokerHash != null && brokerHash.length() > 0 && !brokerHash.equals(localHash)) { + log.info("[{}][{}] Hash mismatch detected (local: {}, broker: {}). Triggering " + + "reconciliation.", patternConsumer.getPattern().inputPattern(), + patternConsumer.getSubscription(), localHash, brokerHash); + appendRecheckOp(); + } + }); break; } case RECHECK: { @@ -187,20 +265,24 @@ synchronized void triggerNextTask() { newTaskFuture = patternConsumer.recheckTopicsChange(); break; } + case WATCH_TOPIC_LIST_SUCCESS: { + WatchTopicListSuccessTask watchTopicListSuccessTask = (WatchTopicListSuccessTask) task; + newTaskFuture = patternConsumer.handleWatchTopicListSuccess(watchTopicListSuccessTask.response, + watchTopicListSuccessTask.localStateTopicsHash, watchTopicListSuccessTask.epoch); + 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()); + log.debug("Pattern consumer [{}] starting task. {}", patternConsumer.getSubscription(), task); } // Trigger next pending task. - taskInProgress = Pair.of(task.getLeft(), newTaskFuture); + taskInProgress = Pair.of(task.type, newTaskFuture); newTaskFuture.thenAccept(ignore -> { if (log.isDebugEnabled()) { - log.debug("Pattern consumer [{}] task finished. {} {} ", patternConsumer.getSubscription(), - task.getLeft(), task.getRight() == null ? "" : task.getRight()); + log.debug("Pattern consumer [{}] task finished. {}", patternConsumer.getSubscription(), task); } triggerNextTask(); }).exceptionally(ex -> { @@ -209,8 +291,8 @@ synchronized void triggerNextTask() { * - 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); + log.error("Pattern consumer [{}] task finished. {}. But it failed", patternConsumer.getSubscription(), + task, ex); // Skip if there is already a recheck task in queue. synchronized (PatternConsumerUpdateQueue.this) { if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { @@ -242,13 +324,13 @@ public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() } private enum UpdateSubscriptionType { - /** A marker that indicates the consumer's subscribe task.**/ + /** 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, + /** Triggered by topic list watcher when topics changed. **/ + TOPICS_CHANGED, /** A fully check for pattern consumer. **/ - RECHECK; + RECHECK, + /** Handle initial watch topic list success response. **/ + WATCH_TOPIC_LIST_SUCCESS; } } 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 8a3798a670cb5..3da14637a1a6f 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 @@ -33,11 +33,16 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import lombok.AccessLevel; +import lombok.Getter; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; +import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.lookup.GetTopicsResult; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -52,15 +57,14 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl watcherFuture = new CompletableFuture<>(); protected NamespaceName namespaceName; private final AtomicInteger recheckPatternEpoch = new AtomicInteger(); - // If recheckPatternTimeout is not null, it means the broker's topic watcher is disabled. - // The client need falls back to the polling model. private volatile Timeout recheckPatternTimeout = null; - private volatile String topicsHash; private PatternConsumerUpdateQueue updateTaskQueue; private volatile boolean closed = false; @@ -69,7 +73,6 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl conf, ExecutorProvider executorProvider, @@ -80,37 +83,45 @@ public PatternMultiTopicsConsumerImpl(TopicsPattern topicsPattern, super(client, conf, executorProvider, subscribeFuture, schema, interceptors, false /* createTopicIfDoesNotExist */); this.topicsPattern = topicsPattern; - this.topicsHash = topicsHash; this.subscriptionMode = subscriptionMode; this.namespaceName = topicsPattern.namespace(); - this.topicsChangeListener = new PatternTopicsChangedListener(); this.updateTaskQueue = new PatternConsumerUpdateQueue(this); if (subscriptionMode == Mode.PERSISTENT) { - long watcherId = client.newTopicListWatcherId(); - topicListWatcher = new TopicListWatcher(updateTaskQueue, client, topicsPattern, watcherId, - namespaceName, topicsHash, watcherFuture, () -> recheckTopicsChangeAfterReconnect()); - watcherFuture - .exceptionally(ex -> { - if (closed) { - log.warn("Pattern consumer [{}] was closed while creating topic list watcher", - conf.getSubscriptionName(), ex); - } else { - log.warn( - "Pattern consumer [{}] unable to create topic list watcher. Falling back to only polling" - + " for new topics", conf.getSubscriptionName(), ex); - this.recheckPatternTimeout = client.timer() - .newTimeout(this, Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); - } - return null; - }); + subscribeFuture.whenComplete((__, exception) -> { + if (!closed && exception == null) { + long watcherId = client.newTopicListWatcherId(); + topicListWatcher = new TopicListWatcher(updateTaskQueue, client, topicsPattern, watcherId, + namespaceName, this::getLocalStateTopicsHash, watcherFuture, + this::getNextRecheckPatternEpoch); + watcherFuture.whenComplete((watcher, ex) -> { + if (closed) { + log.warn("Pattern consumer [{}] was closed while creating topic list watcher", + conf.getSubscriptionName(), ex); + } else if (ex != null) { + if (ex instanceof PulsarClientException.NotAllowedException) { + // create info message when topic watchers aren't supported + log.info("Pattern consumer [{}] unable to create topic list watcher. {}", + conf.getSubscriptionName(), ex.getMessage()); + } else { + log.warn("Pattern consumer [{}] unable to create topic list watcher.", + conf.getSubscriptionName(), ex); + } + } + scheduleRecheckTopics(); + }); + } + }); } else { log.debug("Pattern consumer [{}] not creating topic list watcher for subscription mode {}", conf.getSubscriptionName(), subscriptionMode); topicListWatcher = null; watcherFuture.complete(null); - this.recheckPatternTimeout = client.timer().newTimeout( - this, Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); + subscribeFuture.whenComplete((__, ex) -> { + if (!closed && ex == null) { + scheduleRecheckTopics(); + } + }); } } @@ -136,46 +147,123 @@ public void run(Timeout timeout) throws Exception { } CompletableFuture recheckTopicsChange() { - String pattern = topicsPattern.inputPattern(); - final int epoch = recheckPatternEpoch.incrementAndGet(); - return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) - .thenCompose(getTopicsResult -> { - // If "recheckTopicsChange" has been called more than one times, only make the last one take affects. - // Use "synchronized (recheckPatternTaskBackoff)" instead of - // `synchronized(PatternMultiTopicsConsumerImpl.this)` to avoid locking in a wider range. - synchronized (PatternMultiTopicsConsumerImpl.this) { - if (recheckPatternEpoch.get() > epoch) { - return CompletableFuture.completedFuture(null); - } - if (log.isDebugEnabled()) { - 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 int epoch = getNextRecheckPatternEpoch(); - final List oldTopics = new ArrayList<>(getPartitions()); - return updateSubscriptions(topicsPattern, this::setTopicsHash, getTopicsResult, - topicsChangeListener, oldTopics, subscription); - } - }).thenAccept(__ -> { - if (recheckPatternTimeout != null) { - this.recheckPatternTimeout = client.timer().newTimeout( - this, Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); + CompletableFuture recheckFuture; + // Prefer watcher-based reconcile when a watcher exists and is connected. Fallback to lookup if watcher + // is not available or the watcher-based request fails. + if (supportsTopicListWatcherReconcile()) { + String localStateTopicsHash = getLocalStateTopicsHash(); + recheckFuture = topicListWatcher.reconcile(localStateTopicsHash).thenCompose(response -> { + return handleWatchTopicListSuccess(response, localStateTopicsHash, epoch); + }).handle((res, ex) -> { + if (ex != null) { + // watcher-based reconcile failed -> fall back to lookup-based recheck + return doLookupBasedRecheck(epoch); + } else { + // watcher-based reconcile completed successfully + return CompletableFuture.completedFuture(null); } - }); + }).thenCompose(Function.identity()); + } else { + // Fallback: perform the existing lookup-based recheck + recheckFuture = doLookupBasedRecheck(epoch); + } + + return recheckFuture.handle((__, ex) -> { + if (ex != null) { + log.info("[{}][{}] Pattern consumer failed to recheck topics changes: {}", + getPattern().inputPattern(), getSubscription(), ex.getMessage()); + } + scheduleRecheckTopics(); + return null; + }); + } + + int getNextRecheckPatternEpoch() { + return recheckPatternEpoch.incrementAndGet(); + } + + CompletableFuture handleWatchTopicListSuccess(CommandWatchTopicListSuccess response, + String localStateTopicsHash, int epoch) { + synchronized (PatternMultiTopicsConsumerImpl.this) { + if (recheckPatternEpoch.get() > epoch) { + return CompletableFuture.completedFuture(null); + } + // Build a GetTopicsResult-like object from the watch response + // so we can reuse updateSubscriptions + final List topics = (response != null) + ? response.getTopicsList() + : Collections.emptyList(); + final String hash = (response != null && response.hasTopicsHash()) + ? response.getTopicsHash() + : null; + final boolean changed = !localStateTopicsHash.equals(hash); + final GetTopicsResult getTopicsResult = + new GetTopicsResult(topics, hash, true, changed); + + final List oldTopics = new ArrayList<>(getPartitions()); + return updateSubscriptions(topicsPattern, getTopicsResult, topicsChangeListener, oldTopics, + subscription); + } + } + + boolean supportsTopicListWatcherReconcile() { + return topicListWatcher != null && topicListWatcher.supportsReconcile() && watcherFuture.isDone() + && !watcherFuture.isCompletedExceptionally() && topicListWatcher.isConnected(); + } + + private synchronized void scheduleRecheckTopics() { + if (!closed) { + // cancel previous timeout if it exists + Timeout oldTimeout = this.recheckPatternTimeout; + if (oldTimeout != null) { + // cancel is a no-op if the timeout has already been executed or cancelled + oldTimeout.cancel(); + } + this.recheckPatternTimeout = client.timer().newTimeout(this, + Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); + } + } + + private CompletableFuture doLookupBasedRecheck(final int epoch) { + final String pattern = topicsPattern.inputPattern(); + return client.getLookup() + .getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, getLocalStateTopicsHash()) + .thenCompose(getTopicsResult -> { + // If "recheckTopicsChange" has been called more than one times, only make the last one take + // affects. + // Use "synchronized (recheckPatternTaskBackoff)" instead of + // `synchronized(PatternMultiTopicsConsumerImpl.this)` to avoid locking in a wider range. + synchronized (PatternMultiTopicsConsumerImpl.this) { + if (recheckPatternEpoch.get() > epoch) { + return CompletableFuture.completedFuture(null); + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] get topics under namespace {}, " + + "topics.size: {}, " + + "topicsHash: {}, filtered: {}", + 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<>(getPartitions()); + return updateSubscriptions(topicsPattern, getTopicsResult, topicsChangeListener, oldTopics, + subscription); + } + }); } static CompletableFuture updateSubscriptions(TopicsPattern topicsPattern, - java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, List oldTopics, String subscriptionForLog) { - topicsHashSetter.accept(getTopicsResult.getTopicsHash()); if (!getTopicsResult.isChanged()) { return CompletableFuture.completedFuture(null); } @@ -203,11 +291,6 @@ public TopicsPattern getPattern() { return this.topicsPattern; } - @VisibleForTesting - void setTopicsHash(String topicsHash) { - this.topicsHash = topicsHash; - } - interface TopicsChangedListener { /*** * unsubscribe and delete {@link ConsumerImpl} in the {@link MultiTopicsConsumerImpl#consumers} map in @@ -414,6 +497,15 @@ Timeout getRecheckPatternTimeout() { return recheckPatternTimeout; } + /** + * Get the current topics hash calculated from the pattern consumer's topic list. + * This is used to validate incremental updates against the broker's hash. + */ + @VisibleForTesting + String getLocalStateTopicsHash() { + return TopicList.calculateHash(getPartitions()); + } + protected void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { 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 96086063d7748..874c311f8fd27 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 @@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import lombok.Builder; import lombok.Getter; import org.apache.commons.lang3.tuple.Pair; @@ -658,9 +659,14 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo conf.getSubscriptionName(), namespaceName, topicName)); } - List topicsList = getTopicsResult.getTopics(); + List topicsList; if (!getTopicsResult.isFiltered()) { topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), pattern); + } else { + // deduplicate java.lang.String instances using TopicName's cache + topicsList = getTopicsResult.getTopics().stream() + .map(TopicName::get).map(TopicName::toString) + .collect(Collectors.toList()); } conf.getTopicNames().addAll(topicsList); @@ -672,7 +678,6 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo // Pattern consumer has his unique check mechanism, so do not need the feature "autoUpdatePartitions". conf.setAutoUpdatePartitions(false); ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(pattern, - getTopicsResult.getTopicsHash(), PulsarClientImpl.this, conf, externalExecutorProvider, 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 49cf64656fef7..0c854a068a278 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 @@ -18,14 +18,20 @@ */ package org.apache.pulsar.client.impl; +import com.google.common.annotations.VisibleForTesting; import io.netty.channel.ChannelHandlerContext; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.IntSupplier; +import java.util.function.Supplier; +import lombok.AccessLevel; +import lombok.Getter; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.protocol.Commands; @@ -46,27 +52,27 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. private final String name; private final ConnectionHandler connectionHandler; private final TopicsPattern topicsPattern; + @Getter(value = AccessLevel.PROTECTED, onMethod_ = @VisibleForTesting) private final long watcherId; private volatile long createWatcherDeadline = 0; private final NamespaceName namespace; - // TODO maintain the value based on updates from broker and warn the user if inconsistent with hash from polling - private String topicsHash; + // maintain the value based on updates from broker and allow external components to query it + private final Supplier localStateTopicsHashSupplier; private final CompletableFuture watcherFuture; + private final IntSupplier nextRecheckPatternEpochSupplier; private final AtomicInteger previousExceptionCount = new AtomicInteger(); private final AtomicReference clientCnxUsedForWatcherRegistration = new AtomicReference<>(); - private final Runnable recheckTopicsChangeAfterReconnect; - /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, PulsarClientImpl client, TopicsPattern topicsPattern, long watcherId, - NamespaceName namespace, String topicsHash, + NamespaceName namespace, Supplier localStateTopicsHashSupplier, CompletableFuture watcherFuture, - Runnable recheckTopicsChangeAfterReconnect) { + IntSupplier nextRecheckPatternEpochSupplier) { super(client, topicsPattern.topicLookupNameForTopicListWatcherPlacement()); this.patternConsumerUpdateQueue = patternConsumerUpdateQueue; this.name = "Watcher(" + topicsPattern + ")"; @@ -81,9 +87,9 @@ public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, this.topicsPattern = topicsPattern; this.watcherId = watcherId; this.namespace = namespace; - this.topicsHash = topicsHash; + this.localStateTopicsHashSupplier = localStateTopicsHashSupplier; this.watcherFuture = watcherFuture; - this.recheckTopicsChangeAfterReconnect = recheckTopicsChangeAfterReconnect; + this.nextRecheckPatternEpochSupplier = nextRecheckPatternEpochSupplier; connectionHandler.grabCnx(); } @@ -108,6 +114,15 @@ public boolean connectionFailed(PulsarClientException exception) { @Override public CompletableFuture connectionOpened(ClientCnx cnx) { + if (!cnx.isSupportsTopicWatchers()) { + watcherFuture.completeExceptionally(new PulsarClientException.NotAllowedException( + "Broker does not allow broker side pattern evaluation.")); + setState(State.Closed); + deregisterFromClientCnx(); + connectionClosed(cnx); + return CompletableFuture.completedFuture(null); + } + previousExceptionCount.set(0); State state = getState(); @@ -130,11 +145,10 @@ public CompletableFuture connectionOpened(ClientCnx cnx) { // synchronized this, because redeliverUnAckMessage eliminate the epoch inconsistency between them synchronized (this) { setClientCnx(cnx); - BaseCommand watchRequest = Commands.newWatchTopicList(requestId, watcherId, namespace.toString(), - topicsPattern.inputPattern(), topicsHash); - - cnx.newWatchTopicList(watchRequest, requestId) - + String localStateTopicsHash = localStateTopicsHashSupplier.get(); + int epoch = nextRecheckPatternEpochSupplier.getAsInt(); + cnx.newWatchTopicList(requestId, watcherId, namespace.toString(), topicsPattern.inputPattern(), + localStateTopicsHash) .thenAccept(response -> { synchronized (TopicListWatcher.this) { if (!changeToReadyState()) { @@ -142,14 +156,15 @@ public CompletableFuture connectionOpened(ClientCnx cnx) { // drops the watcher on its side setState(State.Closed); deregisterFromClientCnx(); + log.warn("[{}] Watcher was closed while reconnecting, closing the connection to {}.", + topic, cnx.channel().remoteAddress()); cnx.channel().close(); future.complete(null); return; } } this.connectionHandler.resetBackoff(); - - recheckTopicsChangeAfterReconnect.run(); + patternConsumerUpdateQueue.appendWatchTopicListSuccessOp(response, localStateTopicsHash, epoch); watcherFuture.complete(this); future.complete(null); }).exceptionally((e) -> { @@ -280,7 +295,33 @@ private void cleanupAtClose(CompletableFuture closeFuture, Throwable excep } public void handleCommandWatchTopicUpdate(CommandWatchTopicUpdate update) { - patternConsumerUpdateQueue.appendTopicsRemovedOp(update.getDeletedTopicsList()); - patternConsumerUpdateQueue.appendTopicsAddedOp(update.getNewTopicsList()); + if (update == null) { + return; + } + patternConsumerUpdateQueue.appendTopicsChangedOp(update.getNewTopicsList(), update.getDeletedTopicsList(), + update.hasTopicsHash() ? update.getTopicsHash() : ""); + } + + /** + * Perform a single reconciliation request using the existing watcher id and the watcher's last-known topics hash. + * This will send a WatchTopicList request including the topics-hash to the broker. If the watcher is not connected, + * the returned future will be completed exceptionally. + */ + public CompletableFuture reconcile(String localStateTopicsHash) { + ClientCnx c = cnx(); + if (c == null || !isConnected()) { + CompletableFuture f = new CompletableFuture<>(); + f.completeExceptionally(new IllegalStateException("Watcher is not connected")); + return f; + } + long requestId = client.newRequestId(); + // Use the convenience ClientCnx overload that accepts a topicsHash + return c.newWatchTopicList(requestId, watcherId, namespace.toString(), + topicsPattern.inputPattern(), localStateTopicsHash); + } + + public boolean supportsReconcile() { + ClientCnx cnx = cnx(); + return cnx != null && cnx.isSupportsTopicWatcherReconcile(); } } 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 bcb3791be8505..ae0797fa4939a 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 @@ -61,7 +61,7 @@ class ClientTestFixtures { // return createPulsarClientMock(mock(ExecutorService.class)); // } - static PulsarClientImpl createPulsarClientMock(ExecutorProvider executorProvider, + public static PulsarClientImpl createPulsarClientMock(ExecutorProvider executorProvider, ExecutorService internalExecutorService) { PulsarClientImpl clientMock = mock(PulsarClientImpl.class, Mockito.RETURNS_DEEP_STUBS); @@ -81,11 +81,19 @@ static PulsarClientImpl createPulsarClientMock(ExecutorProvider executorProv static PulsarClientImpl createPulsarClientMockWithMockedClientCnx( ExecutorProvider executorProvider, ExecutorService internalExecutorService) { - return mockClientCnx(createPulsarClientMock(executorProvider, internalExecutorService)); + return createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutorService, mockClientCnx()); } - static PulsarClientImpl mockClientCnx(PulsarClientImpl clientMock) { - ClientCnx clientCnxMock = mockClientCnx(); + static PulsarClientImpl createPulsarClientMockWithMockedClientCnx( + ExecutorProvider executorProvider, + ExecutorService internalExecutorService, + ClientCnx clientCnxMock) { + return clientMockWithClientCnxMock(createPulsarClientMock(executorProvider, internalExecutorService), + clientCnxMock + ); + } + + static PulsarClientImpl clientMockWithClientCnxMock(PulsarClientImpl clientMock, ClientCnx clientCnxMock) { when(clientMock.getConnection(any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); when(clientMock.getConnection(anyString())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); when(clientMock.getConnection(anyString(), anyInt())) 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 index 01f0be6a85ef6..3f2fa81e44f14 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -28,6 +29,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -35,6 +37,7 @@ import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; +import org.mockito.stubbing.Answer; import org.testng.annotations.Test; @Test(groups = "utils") @@ -65,6 +68,7 @@ private QueueInstance createInstance(CompletableFuture customizedRecheckFu } else { when(patternConsumer.getSubscribeFuture()).thenReturn(CompletableFuture.completedFuture(null)); } + when(patternConsumer.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); @@ -72,11 +76,19 @@ private QueueInstance createInstance(CompletableFuture customizedRecheckFu 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); + CompletableFuture ex = FutureUtil.failedFuture(new RuntimeException("Failed topics changed event")); + Answer answer = invocationOnMock -> { + Collection inputCollection = invocationOnMock.getArgument(0, Collection.class); + if (successTopics.containsAll(inputCollection)) { + return customizedPartialUpdateFuture; + } else if (errorTopics.containsAll(inputCollection)) { + return ex; + } else { + throw new RuntimeException("Unexpected topics changed event"); + } + }; + doAnswer(answer).when(topicsChangeListener).onTopicsAdded(anyCollection()); + doAnswer(answer).when(topicsChangeListener).onTopicsRemoved(anyCollection()); } PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, topicsChangeListener); @@ -104,14 +116,14 @@ public void close() { public void testTopicsChangedEvents() { QueueInstance instance = createInstance(); - Collection topics = Arrays.asList("a"); + Collection addedTopics = Arrays.asList("a"); + Collection removedTopics = Arrays.asList("b"); for (int i = 0; i < 10; i++) { - instance.queue.appendTopicsAddedOp(topics); - instance.queue.appendTopicsRemovedOp(topics); + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); } Awaitility.await().untilAsserted(() -> { - verify(instance.mockedListener, times(10)).onTopicsAdded(topics); - verify(instance.mockedListener, times(10)).onTopicsRemoved(topics); + verify(instance.mockedListener, times(10)).onTopicsAdded(addedTopics); + verify(instance.mockedListener, times(10)).onTopicsRemoved(removedTopics); }); // cleanup. @@ -165,11 +177,11 @@ public void testCompositeTasks() { CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); - Collection topics = Arrays.asList("a"); + Collection addedTopics = Arrays.asList("a"); + Collection removedTopics = Arrays.asList("b"); for (int i = 0; i < 10; i++) { instance.queue.appendRecheckOp(); - instance.queue.appendTopicsAddedOp(topics); - instance.queue.appendTopicsRemovedOp(topics); + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); } recheckFuture.complete(null); Awaitility.await().untilAsserted(() -> { @@ -182,8 +194,8 @@ public void testCompositeTasks() { // "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); + verify(instance.mockedListener, times(0)).onTopicsAdded(addedTopics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(removedTopics); }); // cleanup. @@ -193,20 +205,21 @@ public void testCompositeTasks() { @Test public void testErrorTask() { CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); - Collection successTopics = Arrays.asList("a"); + Collection successTopics = Arrays.asList("a", "b"); 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); + Collection addedTopics = Arrays.asList("a"); + Collection removedTopics = Arrays.asList("b"); + + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); + instance.queue.appendTopicsChangedOp(errorTopics, Collections.emptyList(), ""); + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); 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(2)).onTopicsAdded(addedTopics); + verify(instance.mockedListener, times(2)).onTopicsRemoved(removedTopics); verify(instance.mockedListener, times(1)).onTopicsAdded(errorTopics); // After an error task will push a recheck task to offset. verify(instance.mockedConsumer, times(1)).recheckTopicsChange(); @@ -225,18 +238,19 @@ public void testFailedSubscribe() { 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); + Collection addedTopics = Arrays.asList("a"); + Collection removedTopics = Arrays.asList("b"); + + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); + instance.queue.appendTopicsChangedOp(errorTopics, Collections.emptyList(), ""); + instance.queue.appendTopicsChangedOp(addedTopics, removedTopics, ""); // 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(addedTopics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(removedTopics); verify(instance.mockedListener, times(0)).onTopicsAdded(errorTopics); verify(instance.mockedConsumer, times(0)).recheckTopicsChange(); }); 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 4bb541fbd5aa0..dd4bbff757acd 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 @@ -18,43 +18,98 @@ */ package org.apache.pulsar.client.impl; -import static org.mockito.Mockito.any; +import static org.apache.pulsar.client.impl.ClientTestFixtures.createDelayedCompletedFuture; +import static org.apache.pulsar.client.impl.ClientTestFixtures.createPulsarClientMockWithMockedClientCnx; +import static org.apache.pulsar.client.impl.ClientTestFixtures.mockClientCnx; +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.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.common.collect.Sets; +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timeout; +import io.netty.util.Timer; +import io.netty.util.TimerTask; +import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Deque; +import java.util.HashSet; +import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; +import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.lookup.GetTopicsResult; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.topics.TopicList; +import org.apache.pulsar.common.topics.TopicsPattern; import org.apache.pulsar.common.topics.TopicsPatternFactory; +import org.awaitility.Awaitility; +import org.jspecify.annotations.NonNull; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +/** + * Unit Tests of {@link PatternMultiTopicsConsumerImpl}. + */ public class PatternMultiTopicsConsumerImplTest { - private PatternMultiTopicsConsumerImpl.TopicsChangedListener mockListener; - - private Consumer mockTopicsHashSetter; - + private ExecutorProvider executorProvider; + private ExecutorService internalExecutor; + private PulsarClientImpl clientMock; + private ClientCnx cnx; + private Timer timer; @BeforeMethod(alwaysRun = true) public void setUp() { + executorProvider = new ExecutorProvider(1, "PatternMultiTopicsConsumerImplTest"); + internalExecutor = Executors.newSingleThreadScheduledExecutor(); + cnx = mockClientCnx(); + clientMock = createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutor, cnx); + timer = new HashedWheelTimer(); + when(clientMock.timer()).thenReturn(timer); mockListener = mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); when(mockListener.onTopicsAdded(any())).thenReturn(CompletableFuture.completedFuture(null)); when(mockListener.onTopicsRemoved(any())).thenReturn(CompletableFuture.completedFuture(null)); - mockTopicsHashSetter = mock(Consumer.class); + } + @AfterMethod(alwaysRun = true) + public void cleanUp() { + if (executorProvider != null) { + executorProvider.shutdownNow(); + executorProvider = null; + } + if (internalExecutor != null) { + internalExecutor.shutdownNow(); + internalExecutor = null; + } + if (timer != null) { + timer.stop(); + } } @Test public void testChangedUnfilteredResponse() { PatternMultiTopicsConsumerImpl.updateSubscriptions( TopicsPatternFactory.create(Pattern.compile("tenant/my-ns/name-.*")), - mockTopicsHashSetter, new GetTopicsResult(Arrays.asList( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2", @@ -66,14 +121,12 @@ public void testChangedUnfilteredResponse() { "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); verify(mockListener).onTopicsRemoved(Collections.emptySet()); - verify(mockTopicsHashSetter).accept(null); } @Test public void testChangedFilteredResponse() { PatternMultiTopicsConsumerImpl.updateSubscriptions( TopicsPatternFactory.create(Pattern.compile("tenant/my-ns/name-.*")), - mockTopicsHashSetter, new GetTopicsResult(Arrays.asList( "persistent://tenant/my-ns/name-0", "persistent://tenant/my-ns/name-1", @@ -85,14 +138,12 @@ public void testChangedFilteredResponse() { "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); verify(mockListener).onTopicsRemoved(Collections.emptySet()); - verify(mockTopicsHashSetter).accept("TOPICS_HASH"); } @Test public void testUnchangedResponse() { PatternMultiTopicsConsumerImpl.updateSubscriptions( TopicsPatternFactory.create(Pattern.compile("tenant/my-ns/name-.*")), - mockTopicsHashSetter, new GetTopicsResult(Arrays.asList( "persistent://tenant/my-ns/name-0", "persistent://tenant/my-ns/name-1", @@ -102,6 +153,197 @@ public void testUnchangedResponse() { Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener, never()).onTopicsAdded(any()); verify(mockListener, never()).onTopicsRemoved(any()); - verify(mockTopicsHashSetter).accept("TOPICS_HASH"); + } + + @Test + public void testPatternSubscribeAndReconcileLoop() throws Exception { + TopicsPattern topicsPattern = + TopicsPatternFactory.create("persistent://tenant/namespace/.*", TopicsPattern.RegexImplementation.JDK); + ConsumerConfigurationData consumerConfData = createConsumerConfigurationData(); + consumerConfData.setPatternAutoDiscoveryPeriod(1); + + CopyOnWriteArrayList topics = new CopyOnWriteArrayList<>(); + topics.add("persistent://tenant/namespace/topic1"); + doAnswer(invocationOnMock -> { + long requestId = invocationOnMock.getArgument(0); + long watcherId = invocationOnMock.getArgument(1); + String localHash = invocationOnMock.getArgument(4); + CommandWatchTopicListSuccess success = new CommandWatchTopicListSuccess(); + success.setRequestId(requestId); + success.setWatcherId(watcherId); + List topicsCopy = new ArrayList<>(topics); + String calculatedHash = TopicList.calculateHash(topicsCopy); + if (!localHash.equals(calculatedHash)) { + success.addAllTopics(topicsCopy); + } + success.setTopicsHash(calculatedHash); + return CompletableFuture.completedFuture(success); + }).when(cnx).newWatchTopicList(anyLong(), anyLong(), any(), any(), any()); + doReturn(true).when(cnx).isSupportsTopicWatchers(); + doReturn(true).when(cnx).isSupportsTopicWatcherReconcile(); + + PatternMultiTopicsConsumerImpl consumer = + createPatternMultiTopicsConsumer(consumerConfData, topicsPattern); + assertThat(consumer.subscribeFuture).succeedsWithin(Duration.ofSeconds(5)); + assertThat(consumer.getWatcherFuture()).succeedsWithin(Duration.ofSeconds(5)); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1"); + }); + topics.add("persistent://tenant/namespace/topic2"); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2"); + }); + } + + @Test + public void testPatternSubscribeWithoutWatcher() throws Exception { + TopicsPattern topicsPattern = + TopicsPatternFactory.create("persistent://tenant/namespace/.*", TopicsPattern.RegexImplementation.JDK); + ConsumerConfigurationData consumerConfData = createConsumerConfigurationData(); + consumerConfData.setPatternAutoDiscoveryPeriod(1); + + CopyOnWriteArrayList topics = new CopyOnWriteArrayList<>(); + topics.add("persistent://tenant/namespace/topic1"); + LookupService mockLookup = clientMock.getLookup(); + doAnswer(invocationOnMock -> { + String localHash = invocationOnMock.getArgument(3); + List topicsCopy = new ArrayList<>(topics); + String calculatedHash = TopicList.calculateHash(topicsCopy); + boolean changed = false; + if (!localHash.equals(calculatedHash)) { + changed = true; + } + GetTopicsResult result = new GetTopicsResult(topicsCopy, calculatedHash, false, changed); + return CompletableFuture.completedFuture(result); + }).when(mockLookup).getTopicsUnderNamespace(any(), any(), any(), any()); + doReturn(false).when(cnx).isSupportsTopicWatchers(); + + PatternMultiTopicsConsumerImpl consumer = + createPatternMultiTopicsConsumer(consumerConfData, topicsPattern); + assertThat(consumer.subscribeFuture).succeedsWithin(Duration.ofSeconds(5)); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1"); + }); + topics.add("persistent://tenant/namespace/topic2"); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2"); + }); + } + + @Test + public void testPatternSubscribeAndHashHandlingWithChanges() throws Exception { + TopicsPattern topicsPattern = + TopicsPatternFactory.create("persistent://tenant/namespace/.*", TopicsPattern.RegexImplementation.JDK); + ConsumerConfigurationData consumerConfData = createConsumerConfigurationData(); + consumerConfData.setPatternAutoDiscoveryPeriod(5); + Timer timer = mock(Timer.class); + when(clientMock.timer()).thenReturn(timer); + Deque tasks = new ConcurrentLinkedDeque<>(); + doAnswer(invocationOnMock -> { + TimerTask task = invocationOnMock.getArgument(0); + tasks.add(task); + return mock(Timeout.class); + }).when(timer).newTimeout(any(), anyLong(), any()); + CopyOnWriteArrayList topics = new CopyOnWriteArrayList<>(); + topics.add("persistent://tenant/namespace/topic1"); + consumerConfData.setTopicNames(new HashSet<>(topics)); + AtomicInteger invocationCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + invocationCount.incrementAndGet(); + long requestId = invocationOnMock.getArgument(0); + long watcherId = invocationOnMock.getArgument(1); + String localHash = invocationOnMock.getArgument(4); + CommandWatchTopicListSuccess success = new CommandWatchTopicListSuccess(); + success.setRequestId(requestId); + success.setWatcherId(watcherId); + List topicsCopy = new ArrayList<>(topics); + String calculatedHash = TopicList.calculateHash(topicsCopy); + if (!localHash.equals(calculatedHash)) { + throw new RuntimeException("Assuming no changes"); + } + success.setTopicsHash(calculatedHash); + return CompletableFuture.completedFuture(success); + }).when(cnx).newWatchTopicList(anyLong(), anyLong(), any(), any(), any()); + doReturn(true).when(cnx).isSupportsTopicWatchers(); + doReturn(true).when(cnx).isSupportsTopicWatcherReconcile(); + + PatternMultiTopicsConsumerImpl consumer = + createPatternMultiTopicsConsumer(consumerConfData, topicsPattern); + assertThat(consumer.subscribeFuture).succeedsWithin(Duration.ofSeconds(5)); + assertThat(consumer.getWatcherFuture()).succeedsWithin(Duration.ofSeconds(5)); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1"); + }); + runTimerTasks(tasks); + topics.add("persistent://tenant/namespace/topic2"); + CommandWatchTopicUpdate update = new CommandWatchTopicUpdate(); + TopicListWatcher topicListWatcher = consumer.getTopicListWatcher(); + update.setWatcherId(topicListWatcher.getWatcherId()); + update.addNewTopic("persistent://tenant/namespace/topic2"); + update.setTopicsHash(TopicList.calculateHash(topics)); + topicListWatcher.handleCommandWatchTopicUpdate(update); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic1", + "persistent://tenant/namespace/topic2"); + }); + runTimerTasks(tasks); + runTimerTasks(tasks); + assertThat(invocationCount.get()).isEqualTo(4); + CommandWatchTopicUpdate update2 = new CommandWatchTopicUpdate(); + update2.setWatcherId(topicListWatcher.getWatcherId()); + topics.add("persistent://tenant/namespace/topic3"); + update2.addNewTopic("persistent://tenant/namespace/topic3"); + topics.add("persistent://tenant/namespace/topic4"); + update2.addNewTopic("persistent://tenant/namespace/topic4"); + topics.remove("persistent://tenant/namespace/topic1"); + update2.addDeletedTopic("persistent://tenant/namespace/topic1"); + update2.setTopicsHash(TopicList.calculateHash(topics)); + topicListWatcher.handleCommandWatchTopicUpdate(update2); + Awaitility.await().untilAsserted(() -> { + assertThat(consumer.getPartitions()).containsExactlyInAnyOrder("persistent://tenant/namespace/topic2", + "persistent://tenant/namespace/topic3", "persistent://tenant/namespace/topic4"); + }); + assertThat(invocationCount.get()).isEqualTo(4); + runTimerTasks(tasks); + assertThat(invocationCount.get()).isEqualTo(5); + } + + private static void runTimerTasks(Deque tasks) throws Exception { + // first drain the queue to a list to avoid an infinite loop + List taskList = new ArrayList<>(); + while (!tasks.isEmpty()) { + taskList.add(tasks.poll()); + } + // now run the tasks + for (TimerTask task : taskList) { + task.run(mock(Timeout.class)); + } + } + + private PatternMultiTopicsConsumerImpl createPatternMultiTopicsConsumer(TopicsPattern topicsPattern) { + ConsumerConfigurationData consumerConfData = createConsumerConfigurationData(); + return createPatternMultiTopicsConsumer(consumerConfData, topicsPattern); + } + + private static @NonNull ConsumerConfigurationData createConsumerConfigurationData() { + ConsumerConfigurationData consumerConfData = new ConsumerConfigurationData<>(); + consumerConfData.setSubscriptionName("subscriptionName"); + consumerConfData.setAutoUpdatePartitionsIntervalSeconds(0); + return consumerConfData; + } + + private PatternMultiTopicsConsumerImpl createPatternMultiTopicsConsumer( + ConsumerConfigurationData consumerConfData, TopicsPattern topicsPattern) { + int completionDelayMillis = 100; + Schema schema = Schema.BYTES; + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean())) + .thenAnswer(invocation -> createDelayedCompletedFuture( + new PartitionedTopicMetadata(), completionDelayMillis)); + PatternMultiTopicsConsumerImpl consumer = new PatternMultiTopicsConsumerImpl( + topicsPattern, clientMock, consumerConfData, executorProvider, + new CompletableFuture<>(), schema, CommandGetTopicsOfNamespace.Mode.PERSISTENT, null); + return consumer; } } 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 601045d17c147..9dccc7f7c1dff 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 @@ -24,6 +24,7 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; 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.assertTrue; @@ -35,14 +36,13 @@ import java.util.regex.Pattern; import lombok.Cleanup; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl.TopicsChangedListener; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.topics.TopicsPatternFactory; -import org.mockito.ArgumentCaptor; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -53,6 +53,7 @@ public class TopicListWatcherTest { private PulsarClientImpl client; private CompletableFuture watcherFuture; private TopicsChangedListener listener; + private PatternMultiTopicsConsumerImpl patternConsumer; @BeforeMethod(alwaysRun = true) public void setup() { @@ -74,17 +75,23 @@ public void setup() { when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); CompletableFuture completedFuture = CompletableFuture.completedFuture(null); - PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); - when(patternConsumer.getSubscribeFuture()).thenReturn(completedFuture); + patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.getPattern()).thenReturn(TopicsPatternFactory.create(Pattern.compile(topic))); + when(patternConsumer.getPartitions()).thenReturn(Collections.singletonList("persistent://tenant/ns/topic11")); + CompletableFuture> subscribeFuture = CompletableFuture.completedFuture(patternConsumer); + when(patternConsumer.getSubscribeFuture()).thenReturn(subscribeFuture); when(patternConsumer.recheckTopicsChange()).thenReturn(completedFuture); when(listener.onTopicsAdded(anyCollection())).thenReturn(completedFuture); when(listener.onTopicsRemoved(anyCollection())).thenReturn(completedFuture); + when(patternConsumer.handleWatchTopicListSuccess(any(), any(), anyInt())).thenReturn(completedFuture); + when(patternConsumer.supportsTopicListWatcherReconcile()).thenReturn(true); PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, listener); watcherFuture = new CompletableFuture<>(); watcher = new TopicListWatcher(queue, client, TopicsPatternFactory.create(Pattern.compile(topic)), 7, - NamespaceName.get("tenant/ns"), null, watcherFuture, () -> {}); + NamespaceName.get("tenant/ns"), patternConsumer::getLocalStateTopicsHash, watcherFuture, + () -> 0); } @Test @@ -94,34 +101,44 @@ public void testWatcherGrabsConnection() { @Test public void testWatcherCreatesBrokerSideObjectWhenConnected() { - ClientCnx clientCnx = mock(ClientCnx.class); + ClientCnx clientCnx = mockClientCnx(); CompletableFuture responseFuture = new CompletableFuture<>(); - ArgumentCaptor commandCaptor = ArgumentCaptor.forClass(BaseCommand.class); - when(clientCnx.newWatchTopicList(any(BaseCommand.class), anyLong())).thenReturn(responseFuture); + when(clientCnx.newWatchTopicList(anyLong(), anyLong(), anyString(), anyString(), any())) + .thenReturn(responseFuture); when(clientCnx.ctx()).thenReturn(mock(ChannelHandlerContext.class)); clientCnxFuture.complete(clientCnx); - verify(clientCnx).newWatchTopicList(commandCaptor.capture(), anyLong()); + + verify(clientCnx).newWatchTopicList(anyLong(), anyLong(), anyString(), anyString(), any()); + CommandWatchTopicListSuccess success = new CommandWatchTopicListSuccess() .setWatcherId(7) - .setRequestId(commandCaptor.getValue().getWatchTopicList().getRequestId()) + .setRequestId(0) .setTopicsHash("FEED"); success.addTopic("persistent://tenant/ns/topic11"); responseFuture.complete(success); assertTrue(watcherFuture.isDone() && !watcherFuture.isCompletedExceptionally()); } + private static ClientCnx mockClientCnx() { + ClientCnx clientCnx = mock(ClientCnx.class); + when(clientCnx.isSupportsTopicWatchers()).thenReturn(true); + when(clientCnx.isSupportsTopicWatcherReconcile()).thenReturn(true); + return clientCnx; + } + @Test public void testWatcherCallsListenerOnUpdate() { - ClientCnx clientCnx = mock(ClientCnx.class); + ClientCnx clientCnx = mockClientCnx(); CompletableFuture responseFuture = new CompletableFuture<>(); - ArgumentCaptor commandCaptor = ArgumentCaptor.forClass(BaseCommand.class); - when(clientCnx.newWatchTopicList(any(BaseCommand.class), anyLong())).thenReturn(responseFuture); + when(clientCnx.newWatchTopicList(anyLong(), anyLong(), anyString(), anyString(), any())) + .thenReturn(responseFuture); when(clientCnx.ctx()).thenReturn(mock(ChannelHandlerContext.class)); clientCnxFuture.complete(clientCnx); - verify(clientCnx).newWatchTopicList(commandCaptor.capture(), anyLong()); + verify(clientCnx).newWatchTopicList(anyLong(), anyLong(), anyString(), anyString(), any()); + CommandWatchTopicListSuccess success = new CommandWatchTopicListSuccess() .setWatcherId(7) - .setRequestId(commandCaptor.getValue().getWatchTopicList().getRequestId()) + .setRequestId(0) .setTopicsHash("FEED"); success.addTopic("persistent://tenant/ns/topic11"); responseFuture.complete(success); @@ -134,4 +151,38 @@ public void testWatcherCallsListenerOnUpdate() { watcher.handleCommandWatchTopicUpdate(update); verify(listener).onTopicsAdded(Collections.singletonList("persistent://tenant/ns/topic12")); } + + @Test + public void testWatcherTriggersReconciliationOnHashMismatch() { + ClientCnx clientCnx = mockClientCnx(); + + CompletableFuture responseFuture = new CompletableFuture<>(); + when(clientCnx.newWatchTopicList(anyLong(), anyLong(), anyString(), anyString(), any())) + .thenReturn(responseFuture); + when(clientCnx.ctx()).thenReturn(mock(ChannelHandlerContext.class)); + clientCnxFuture.complete(clientCnx); + + CommandWatchTopicListSuccess success = new CommandWatchTopicListSuccess() + .setWatcherId(7) + .setRequestId(0) + .setTopicsHash("FEED"); + success.addTopic("persistent://tenant/ns/topic11"); + responseFuture.complete(success); + + // verify that the response was handled + verify(patternConsumer, times(1)).handleWatchTopicListSuccess(any(), any(), anyInt()); + // sync local hash + when(patternConsumer.getLocalStateTopicsHash()).thenReturn("FEED"); + + // Send update with a mismatching hash + CommandWatchTopicUpdate update = new CommandWatchTopicUpdate() + .setTopicsHash("WRONG_HASH") + .setWatcherId(7) + .addAllNewTopics(Collections.singleton("persistent://tenant/ns/topic12")); + + watcher.handleCommandWatchTopicUpdate(update); + + // Verify that reconciliation was triggered again due to hash mismatch + verify(patternConsumer, times(1)).recheckTopicsChange(); + } } 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 bdbaacc8a9412..2f45298203332 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 @@ -309,6 +309,8 @@ public static BaseCommand newConnectedCommand(int clientProtocolVersion, int max connected.setFeatureFlags().setSupportsTopicWatchers(supportsTopicWatchers); connected.setFeatureFlags().setSupportsGetPartitionedMetadataWithoutAutoCreation(true); + connected.setFeatureFlags().setSupportsReplDedupByLidAndEid(true); + connected.setFeatureFlags().setSupportsTopicWatcherReconcile(supportsTopicWatchers); return cmd; } 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 0e38f0224728d..3e10c1a1981f5 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 @@ -48,7 +48,7 @@ public static List filterTopics(List original, java.util.regex.P } // get topics that match 'topicsPattern' from original topics list - // return result should contain only topic names, without partition part + // matching is performed with base topic names, without partition part public static List filterTopics(List original, String regex, TopicsPattern.RegexImplementation topicsPatternImplementation) { return filterTopics(original, TopicsPatternFactory.create(regex, topicsPatternImplementation)); diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index 19658c5e57ff9..c3fc34c55a1ec 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -302,6 +302,8 @@ message FeatureFlags { 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]; + optional bool supports_repl_dedup_by_lid_and_eid = 6 [default = false]; + optional bool supports_topic_watcher_reconcile = 7 [default = false]; } message CommandConnected { @@ -640,7 +642,7 @@ message CommandTopicMigrated { required ResourceType resource_type = 2; optional string brokerServiceUrl = 3; optional string brokerServiceUrlTls = 4; - + } @@ -1135,6 +1137,6 @@ message BaseCommand { optional CommandWatchTopicListSuccess watchTopicListSuccess = 65; optional CommandWatchTopicUpdate watchTopicUpdate = 66; optional CommandWatchTopicListClose watchTopicListClose = 67; - + optional CommandTopicMigrated topicMigrated = 68; } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConProxyTest.java index cec3392b3859a..f20216d57d8c8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/client/impl/AutoCloseUselessClientConProxyTest.java @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -41,6 +42,7 @@ import org.testng.annotations.Test; @Test +@Slf4j public class AutoCloseUselessClientConProxyTest extends AutoCloseUselessClientConSupports { private static final String TOPIC_NAME = BrokerTestUtil.newUniqueName("pattern_"); private static final String TOPIC_FULL_NAME = "persistent://public/default/" + TOPIC_NAME; @@ -108,6 +110,9 @@ public void testConnectionAutoReleaseWhileUsingTopicsPatternConsumerAndProxy() t .subscriptionName("my-subscription-y") .subscribe(); + waitForTopicListWatcherStarted(consumer); + waitForTopicListWatcherStarted(consumer2); + int poolSizeAfterCreatingConsumersAndProducer = pulsarClient.getCnxPool().getPoolSize(); // check that there are more than 3 connections // at least 3 connections are required: From bd6542c0b52a82d285431ac97db5c7be38c293ce Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Wed, 11 Feb 2026 10:43:45 +0530 Subject: [PATCH 30/53] [fix] Update bookkeeper version for asyncOpenLedger() changes --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- pom.xml | 6 +- 2 files changed, 31 insertions(+), 31 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 78a02b769a3ff..408c26dbdcf4b 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -357,34 +357,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 - - com.datastax.oss-bookkeeper-common-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-common-allocator-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-proto-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-server-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-tools-framework-4.17.1.0.0.4.jar - - com.datastax.oss-circe-checksum-4.17.1.0.0.4.jar - - com.datastax.oss-cpu-affinity-4.17.1.0.0.4.jar - - com.datastax.oss-statelib-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-api-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-common-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-java-client-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-java-client-base-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-proto-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-server-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-service-api-4.17.1.0.0.4.jar - - com.datastax.oss-stream-storage-service-impl-4.17.1.0.0.4.jar - - com.datastax.oss-http-server-4.17.1.0.0.4.jar - - com.datastax.oss-vertx-http-server-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-stats-api-4.17.1.0.0.4.jar - - com.datastax.oss-prometheus-metrics-provider-4.17.1.0.0.4.jar - - com.datastax.oss-distributedlog-common-4.17.1.0.0.4.jar - - com.datastax.oss-distributedlog-core-4.17.1.0.0.4-tests.jar - - com.datastax.oss-distributedlog-core-4.17.1.0.0.4.jar - - com.datastax.oss-distributedlog-protocol-4.17.1.0.0.4.jar - - com.datastax.oss-codahale-metrics-provider-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-slogger-api-4.17.1.0.0.4.jar - - com.datastax.oss-bookkeeper-slogger-slf4j-4.17.1.0.0.4.jar - - com.datastax.oss-native-io-4.17.1.0.0.4.jar + - com.datastax.oss-bookkeeper-common-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-common-allocator-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-proto-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-server-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-tools-framework-4.17.1.0.0.5.jar + - com.datastax.oss-circe-checksum-4.17.1.0.0.5.jar + - com.datastax.oss-cpu-affinity-4.17.1.0.0.5.jar + - com.datastax.oss-statelib-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-api-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-common-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-java-client-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-java-client-base-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-proto-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-server-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-service-api-4.17.1.0.0.5.jar + - com.datastax.oss-stream-storage-service-impl-4.17.1.0.0.5.jar + - com.datastax.oss-http-server-4.17.1.0.0.5.jar + - com.datastax.oss-vertx-http-server-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-stats-api-4.17.1.0.0.5.jar + - com.datastax.oss-prometheus-metrics-provider-4.17.1.0.0.5.jar + - com.datastax.oss-distributedlog-common-4.17.1.0.0.5.jar + - com.datastax.oss-distributedlog-core-4.17.1.0.0.5-tests.jar + - com.datastax.oss-distributedlog-core-4.17.1.0.0.5.jar + - com.datastax.oss-distributedlog-protocol-4.17.1.0.0.5.jar + - com.datastax.oss-codahale-metrics-provider-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-slogger-api-4.17.1.0.0.5.jar + - com.datastax.oss-bookkeeper-slogger-slf4j-4.17.1.0.0.5.jar + - com.datastax.oss-native-io-4.17.1.0.0.5.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/pom.xml b/pom.xml index 19fdf999368a1..aa2c86254c9e3 100644 --- a/pom.xml +++ b/pom.xml @@ -177,8 +177,8 @@ flexible messaging model and an intuitive client API. 1.28.0 - 4.17.1.0.0.4 - 4.17.2 + 4.17.1.0.0.5 + 4.17.3 3.9.4 1.11.0 1.14.0 @@ -285,7 +285,7 @@ flexible messaging model and an intuitive client API. 2.0.2 5.12.1 18.0.0 - 0.9.4 + 0.9.6 5.3.1 3.16.3 From 2d9eb62608c0076d7e3953f214cee4429c40de1a Mon Sep 17 00:00:00 2001 From: guptas6est Date: Fri, 6 Feb 2026 15:31:09 +0530 Subject: [PATCH 31/53] [fix][sec] Upgrade OpenSearch to 2.19.4 to remediate CVE-2025-9624 (#25206) (cherry picked from commit 64333094705769640d747b868017b01931b17705) (cherry picked from commit 64f6407bb0a562620e3ee5cbb8567bc4f8ec3916) From 41ebd85ab720ff213cc0bdc0307acd354216bd60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?B=C3=A4m?= Date: Fri, 6 Feb 2026 12:15:52 +0100 Subject: [PATCH 32/53] [feat][io] implement pip-297 for jdbc sinks (#25195) (cherry picked from commit 6f4ac21eeed7bf9a8d4306c17468ce55b74f84a1) (cherry picked from commit 998a4b1677b73c24de1af23931c02badbd9b57a0) --- .../pulsar/io/jdbc/JdbcAbstractSink.java | 25 ++++++- .../pulsar/io/jdbc/SqliteJdbcSinkTest.java | 74 ++++++++++++++++++- 2 files changed, 97 insertions(+), 2 deletions(-) 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 ca33b3cfdaba9..73ba6b712f022 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 @@ -35,6 +35,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; import lombok.AllArgsConstructor; @@ -50,6 +51,11 @@ */ @Slf4j public abstract class JdbcAbstractSink implements Sink { + + private enum State { + OPEN, FAILED, CLOSED + } + // ----- Runtime fields protected JdbcSinkConfig jdbcSinkConfig; @Getter @@ -73,9 +79,12 @@ public abstract class JdbcAbstractSink implements Sink { private AtomicBoolean isFlushing; private int batchSize; private ScheduledExecutorService flushExecutor; + private SinkContext sinkContext; + private final AtomicReference state = new AtomicReference<>(State.OPEN); @Override public void open(Map config, SinkContext sinkContext) throws Exception { + this.sinkContext = sinkContext; jdbcSinkConfig = JdbcSinkConfig.load(config, sinkContext); jdbcSinkConfig.validate(); @@ -148,6 +157,7 @@ private static List getListFromConfig(String jdbcSinkConfig) { @Override public void close() throws Exception { + state.set(State.CLOSED); if (flushExecutor != null) { int timeoutMs = jdbcSinkConfig.getTimeoutMs() * 2; flushExecutor.shutdown(); @@ -310,8 +320,9 @@ private void flush() { connection.rollback(); } } catch (Exception ex) { - throw new RuntimeException(ex); + log.error("Failed to rollback transaction", ex); } + fatal(e); } isFlushing.set(false); @@ -385,4 +396,16 @@ private static boolean isBatchItemFailed(int returnCode) { return true; } + /** + * Signal a fatal exception to the framework. + * This will cause the function instance to terminate properly. + * + * @param e the fatal exception + */ + private void fatal(Exception e) { + if (sinkContext != null && state.compareAndSet(State.OPEN, State.FAILED)) { + sinkContext.fatal(e); + } + } + } 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 901ac9f1e392f..7cab33df3099b 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 @@ -18,11 +18,16 @@ */ package org.apache.pulsar.io.jdbc; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import java.sql.PreparedStatement; +import java.sql.SQLException; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -56,6 +61,7 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.source.PulsarRecord; +import org.apache.pulsar.io.core.SinkContext; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -133,7 +139,9 @@ protected void configure(Map configuration) { @AfterMethod(alwaysRun = true) public void tearDown() throws Exception { - jdbcSink.close(); + if (jdbcSink != null) { + jdbcSink.close(); + } sqliteUtils.tearDown(); } @@ -860,6 +868,70 @@ public void testNullValueAction(NullValueActionTestConfig config) throws Excepti } } + /** + * Test that fatal() is called when an unrecoverable exception occurs during flush. + * This verifies the PIP-297 implementation for proper termination of the sink. + * + * The test works by: + * 1. Opening the sink with a valid table (so open() succeeds) + * 2. Using reflection to replace the insertStatement with a mock that throws SQLException + * 3. Writing a record to trigger flush + * 4. Verifying that fatal() was called with the exception + */ + @Test + public void testFatalCalledOnFlushException() throws Exception { + jdbcSink.close(); + jdbcSink = null; + + String jdbcUrl = sqliteUtils.sqliteUri(); + Map conf = Maps.newHashMap(); + conf.put("jdbcUrl", jdbcUrl); + conf.put("tableName", tableName); // Use valid table so open() succeeds + conf.put("key", "field3"); + conf.put("nonKey", "field1,field2"); + conf.put("batchSize", 1); + + SinkContext mockSinkContext = mock(SinkContext.class); + AtomicReference fatalException = new AtomicReference<>(); + doAnswer(invocation -> { + fatalException.set(invocation.getArgument(0)); + return null; + }).when(mockSinkContext).fatal(any(Throwable.class)); + + SqliteJdbcAutoSchemaSink sinkWithContext = new SqliteJdbcAutoSchemaSink(); + try { + sinkWithContext.open(conf, mockSinkContext); + + // Create a mock PreparedStatement that throws SQLException on execute() + PreparedStatement mockStatement = mock(PreparedStatement.class); + SQLException simulatedException = new SQLException("Simulated database connection failure"); + doThrow(simulatedException).when(mockStatement).execute(); + doThrow(simulatedException).when(mockStatement).executeBatch(); + + // Use reflection to replace the insertStatement with our mock + FieldUtils.writeField(sinkWithContext, "insertStatement", mockStatement, true); + + Foo insertObj = new Foo("f1", "f2", 1); + Map props = Maps.newHashMap(); + props.put("ACTION", "INSERT"); + CompletableFuture future = new CompletableFuture<>(); + sinkWithContext.write(createMockFooRecord(insertObj, props, future)); + + // Wait for the flush to complete and fail + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + verify(mockSinkContext).fatal(any(Throwable.class)); + Assert.assertNotNull(fatalException.get()); + Assert.assertTrue(fatalException.get() instanceof SQLException); + Assert.assertEquals(fatalException.get().getMessage(), "Simulated database connection failure"); + }); + + // Verify the record was failed (not acked) + Assert.assertFalse(future.get(1, TimeUnit.SECONDS)); + } finally { + sinkWithContext.close(); + } + } + @SuppressWarnings("unchecked") private Record createMockFooRecord(Foo record, Map actionProperties, CompletableFuture future) { From f4b3fda719aafe916776ad1f8670fdd49ba6b8bb Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 6 Feb 2026 20:28:25 +0800 Subject: [PATCH 33/53] [fix][broker] Fix httpProxyTimeout config (#25223) Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> (cherry picked from commit 2d6ef6f288b7a105c59c176f59d5b3705d106e1b) (cherry picked from commit 3b39c7b52efeb0b1e17212697ade9f33481ba4b8) --- .../proxy/server/AdminProxyHandler.java | 10 ++++++--- .../proxy/server/AdminProxyHandlerTest.java | 22 ++++++++++++++++--- 2 files changed, 26 insertions(+), 6 deletions(-) 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 08f87f3d08a29..090f2f4a384b5 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 @@ -109,7 +109,6 @@ class AdminProxyHandler extends ProxyServlet { TimeUnit.SECONDS); } } - super.setTimeout(config.getHttpProxyTimeout()); } @Override @@ -119,10 +118,15 @@ protected HttpClient createHttpClient() throws ServletException { return httpClient; } - private void customizeHttpClient(HttpClient httpClient) { + protected void customizeHttpClient(HttpClient httpClient) { httpClient.setFollowRedirects(true); + ProtocolHandlers protocolHandlers = httpClient.getProtocolHandlers(); - protocolHandlers.put(new RedirectProtocolHandler(httpClient)); + if (protocolHandlers != null) { + protocolHandlers.put(new RedirectProtocolHandler(httpClient)); + } + + setTimeout(config.getHttpProxyTimeout()); } // This class allows the request body to be replayed, the default implementation 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 fdf9242c9f3d8..151ffc79f0ba2 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 @@ -20,7 +20,9 @@ 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.Assert.assertEquals; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.InputStream; @@ -59,6 +61,20 @@ protected HttpClient createHttpClient() throws ServletException { adminProxyHandler.init(servletConfig); } + @Test + public void testRequestTimeout() { + ProxyConfiguration proxyConfiguration = spy(new ProxyConfiguration()); + proxyConfiguration.setHttpProxyTimeout(120 * 1000); + + adminProxyHandler = new AdminProxyHandler(proxyConfiguration, + mock(BrokerDiscoveryProvider.class), mock(Authentication.class)); + + HttpClient httpClient = mock(HttpClient.class); + adminProxyHandler.customizeHttpClient(httpClient); + + assertEquals(adminProxyHandler.getTimeout(), 120 * 1000); + } + @Test public void replayableProxyContentProviderTest() throws Exception { HttpServletRequest request = mock(HttpServletRequest.class); @@ -102,10 +118,10 @@ adminProxyHandler.new ReplayableProxyContentProvider(request, mock(HttpServletRe } // then - Assert.assertEquals(consumedBytes, requestBodySize); + assertEquals(consumedBytes, requestBodySize); Field field = replayableProxyContentProvider.getClass().getDeclaredField("bodyBufferMaxSizeReached"); field.setAccessible(true); - Assert.assertEquals(((boolean) field.get(replayableProxyContentProvider)), true); + assertEquals(((boolean) field.get(replayableProxyContentProvider)), true); } @Test @@ -138,7 +154,7 @@ adminProxyHandler.new ReplayableProxyContentProvider(request, mock(HttpServletRe byte[] consumedBytes = new byte[consumeBuffer.limit()]; consumeBuffer.get(consumedBytes); // then - Assert.assertEquals(consumedBytes, inputBuffer); + assertEquals(consumedBytes, inputBuffer, "i=" + i); } } } From f2a76477560e5ea93be0810df3255b9ccedc15fb Mon Sep 17 00:00:00 2001 From: Omar Yasin Date: Fri, 6 Feb 2026 04:46:26 -0800 Subject: [PATCH 34/53] [improve][broker] Add strictAuthMethod to require explicit authentication method (#25185) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ómar K. Yasin (cherry picked from commit bae91734a8584e7610c3945351d82ba52f0c9fae) (cherry picked from commit 27e34f666d093327bf894668a286a0e037245cc0) --- .../pulsar/broker/ServiceConfiguration.java | 8 +++++ .../authentication/AuthenticationService.java | 8 +++++ .../auth/AuthenticationServiceTest.java | 30 +++++++++++++++++++ 3 files changed, 46 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 4b4c0d787d49a..9f20b6a30402a 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 @@ -1742,6 +1742,14 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Enable authentication" ) private boolean authenticationEnabled = false; + + @FieldContext( + category = CATEGORY_AUTHENTICATION, + doc = "Strictly enforce authentication method. If specified, Pulsar will only attempt to authenticate with " + + "the provided method. If no method is provided, authentication fails." + ) + private boolean strictAuthMethod = false; + @FieldContext( category = CATEGORY_AUTHENTICATION, doc = "Authentication provider name list, which is a list of class names" diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java index e2bf4dcc0156d..5b719bd680145 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java @@ -47,6 +47,7 @@ public class AuthenticationService implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(AuthenticationService.class); private final String anonymousUserRole; + private final boolean strictAuthMethod; private final Map providers = new LinkedHashMap<>(); @@ -57,6 +58,7 @@ public AuthenticationService(ServiceConfiguration conf) throws PulsarServerExcep public AuthenticationService(ServiceConfiguration conf, OpenTelemetry openTelemetry) throws PulsarServerException { anonymousUserRole = conf.getAnonymousUserRole(); + strictAuthMethod = conf.isStrictAuthMethod(); if (conf.isAuthenticationEnabled()) { try { Map> providerMap = new LinkedHashMap<>(); @@ -138,6 +140,12 @@ public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletRe throw e; } } else { + if (strictAuthMethod) { + if (LOG.isDebugEnabled()) { + LOG.debug("No authentication method provided while one was is required"); + } + throw new AuthenticationException("Authentication method missing"); + } for (AuthenticationProvider provider : providers.values()) { try { return provider.authenticateHttpRequest(request, response); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java index c4e149eb1b83c..3c90d573919ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthenticationServiceTest.java @@ -221,6 +221,36 @@ public void testHttpRequestWithMultipleProviders() throws Exception { (AuthenticationDataSource) null)).isEqualTo("role2"); } + @Test(timeOut = 10000) + public void testStrictAuthMethodEnforcement() throws Exception { + ServiceConfiguration config = new ServiceConfiguration(); + Set providersClassNames = Sets.newHashSet(MockAuthenticationProvider.class.getName()); + config.setAuthenticationProviders(providersClassNames); + config.setAuthenticationEnabled(true); + config.setStrictAuthMethod(true); + @Cleanup + AuthenticationService service = new AuthenticationService(config); + + // Test: Request without auth method header should fail when strictAuthMethod is enabled + HttpServletRequest requestWithoutAuthMethod = mock(HttpServletRequest.class); + when(requestWithoutAuthMethod.getRemoteAddr()).thenReturn("192.168.1.1"); + when(requestWithoutAuthMethod.getRemotePort()).thenReturn(8080); + // No X-Pulsar-Auth-Method-Name header set + + assertThatThrownBy(() -> service.authenticateHttpRequest(requestWithoutAuthMethod, (HttpServletResponse) null)) + .isInstanceOf(AuthenticationException.class) + .hasMessage("Authentication method missing"); + + // Test: Request with auth method header should still succeed + HttpServletRequest requestWithAuthMethod = mock(HttpServletRequest.class); + when(requestWithAuthMethod.getRemoteAddr()).thenReturn("192.168.1.1"); + when(requestWithAuthMethod.getRemotePort()).thenReturn(8080); + when(requestWithAuthMethod.getHeader("X-Pulsar-Auth-Method-Name")).thenReturn("auth"); + + boolean result = service.authenticateHttpRequest(requestWithAuthMethod, (HttpServletResponse) null); + assertTrue(result, "Authentication should succeed when auth method is provided"); + } + public static class MockHttpAuthenticationProvider implements AuthenticationProvider { @Override public void close() throws IOException { From cfa360d5aa0b4d8e15b43aaa1e83046af1153354 Mon Sep 17 00:00:00 2001 From: gulecroc Date: Tue, 11 Nov 2025 16:20:42 +0100 Subject: [PATCH 35/53] [feat][client] oauth2 trustcerts file and timeouts (#24944) (cherry picked from commit b789d825feb3975092b2da59dea10e512d5d0b42) (cherry picked from commit f8827bd9f0069bab746e3ed5cc1046adcd007d90) --- .../oauth2/AuthenticationFactoryOAuth2.java | 154 ++++++++++++++++-- .../auth/oauth2/ClientCredentialsFlow.java | 110 +++++++------ .../client/impl/auth/oauth2/FlowBase.java | 76 ++++++++- .../protocol/DefaultMetadataResolver.java | 95 +++++------ .../auth/oauth2/protocol/TokenClient.java | 59 +++---- .../AuthenticationFactoryOAuth2Test.java | 59 +++++++ 6 files changed, 395 insertions(+), 158 deletions(-) create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java index c9abb3a3c0147..033d5308a2a96 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java @@ -20,6 +20,7 @@ import java.net.URL; import java.time.Clock; +import java.time.Duration; import org.apache.pulsar.client.api.Authentication; /** @@ -31,9 +32,9 @@ public final class AuthenticationFactoryOAuth2 { /** * Authenticate with client credentials. * - * @param issuerUrl the issuer URL + * @param issuerUrl the issuer URL * @param credentialsUrl the credentials URL - * @param audience An optional field. The audience identifier used by some Identity Providers, like Auth0. + * @param audience An optional field. The audience identifier used by some Identity Providers, like Auth0. * @return an Authentication object */ public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl, String audience) { @@ -43,23 +44,144 @@ public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl /** * Authenticate with client credentials. * - * @param issuerUrl the issuer URL + * @param issuerUrl the issuer URL * @param credentialsUrl the credentials URL - * @param audience An optional field. The audience identifier used by some Identity Providers, like Auth0. - * @param scope An optional field. The value of the scope parameter is expressed as a list of space-delimited, - * case-sensitive strings. The strings are defined by the authorization server. - * If the value contains multiple space-delimited strings, their order does not matter, - * and each string adds an additional access range to the requested scope. - * From here: https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.2 + * @param audience An optional field. The audience identifier used by some Identity Providers, like Auth0. + * @param scope An optional field. The value of the scope parameter is expressed as a list of + * space-delimited, + * case-sensitive strings. The strings are defined by the authorization server. + * If the value contains multiple space-delimited strings, their order does not matter, + * and each string adds an additional access range to the requested scope. + * From here: https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.2 * @return an Authentication object */ public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl, String audience, String scope) { - ClientCredentialsFlow flow = ClientCredentialsFlow.builder() - .issuerUrl(issuerUrl) - .privateKey(credentialsUrl.toExternalForm()) - .audience(audience) - .scope(scope) - .build(); - return new AuthenticationOAuth2(flow, Clock.systemDefaultZone()); + return clientCredentialsBuilder().issuerUrl(issuerUrl).credentialsUrl(credentialsUrl).audience(audience) + .scope(scope).build(); } + + /** + * A builder to create an authentication with client credentials. + * + * @return the builder + */ + public static ClientCredentialsBuilder clientCredentialsBuilder() { + return new ClientCredentialsBuilder(); + } + + public static class ClientCredentialsBuilder { + + private URL issuerUrl; + private URL credentialsUrl; + private String audience; + private String scope; + private Duration connectTimeout; + private Duration readTimeout; + private String trustCertsFilePath; + + private ClientCredentialsBuilder() { + } + + /** + * Required issuer URL. + * + * @param issuerUrl the issuer URL + * @return the builder + */ + public ClientCredentialsBuilder issuerUrl(URL issuerUrl) { + this.issuerUrl = issuerUrl; + return this; + } + + /** + * Required credentials URL. + * + * @param credentialsUrl the credentials URL + * @return the builder + */ + public ClientCredentialsBuilder credentialsUrl(URL credentialsUrl) { + this.credentialsUrl = credentialsUrl; + return this; + } + + /** + * Optional audience identifier used by some Identity Providers, like Auth0. + * + * @param audience the audiance + * @return the builder + */ + public ClientCredentialsBuilder audience(String audience) { + this.audience = audience; + return this; + } + + /** + * Optional scope expressed as a list of space-delimited, case-sensitive strings. + * The strings are defined by the authorization server. + * If the value contains multiple space-delimited strings, their order does not matter, + * and each string adds an additional access range to the requested scope. + * From here: https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.2 + * + * @param scope the scope + * @return the builder + */ + public ClientCredentialsBuilder scope(String scope) { + this.scope = scope; + return this; + } + + /** + * Optional HTTP connection timeout. + * + * @param connectTimeout the connect timeout + * @return the builder + */ + public ClientCredentialsBuilder connectTimeout(Duration connectTimeout) { + this.connectTimeout = connectTimeout; + return this; + } + + /** + * Optional HTTP read timeout. + * + * @param readTimeout the read timeout + * @return the builder + */ + public ClientCredentialsBuilder readTimeout(Duration readTimeout) { + this.readTimeout = readTimeout; + return this; + } + + /** + * Optional path to the file containing the trusted certificate(s) of the token issuer. + * + * @param trustCertsFilePath the path to the file containing the trusted certificate(s) + * @return the builder + */ + public ClientCredentialsBuilder trustCertsFilePath(String trustCertsFilePath) { + this.trustCertsFilePath = trustCertsFilePath; + return this; + } + + /** + * Authenticate with client credentials. + * + * @return an Authentication object + */ + public Authentication build() { + ClientCredentialsFlow flow = ClientCredentialsFlow.builder() + .issuerUrl(issuerUrl) + .privateKey(credentialsUrl == null ? null : credentialsUrl.toExternalForm()) + .audience(audience) + .scope(scope) + .connectTimeout(connectTimeout) + .readTimeout(readTimeout) + .trustCertsFilePath(trustCertsFilePath) + .build(); + return new AuthenticationOAuth2(flow, Clock.systemDefaultZone()); + } + + } + + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java index ef10f1afdb63b..7f64c0b18ac73 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java @@ -26,6 +26,7 @@ import java.net.URL; import java.net.URLConnection; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.Map; import lombok.Builder; import lombok.extern.slf4j.Slf4j; @@ -60,62 +61,17 @@ class ClientCredentialsFlow extends FlowBase { private boolean initialized = false; @Builder - public ClientCredentialsFlow(URL issuerUrl, String audience, String privateKey, String scope) { - super(issuerUrl); + public ClientCredentialsFlow(URL issuerUrl, String audience, String privateKey, String scope, + Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) { + super(issuerUrl, connectTimeout, readTimeout, trustCertsFilePath); this.audience = audience; this.privateKey = privateKey; this.scope = scope; } - @Override - public void initialize() throws PulsarClientException { - super.initialize(); - assert this.metadata != null; - - URL tokenUrl = this.metadata.getTokenEndpoint(); - this.exchanger = new TokenClient(tokenUrl); - initialized = true; - } - - public TokenResult authenticate() throws PulsarClientException { - // read the private key from storage - KeyFile keyFile; - try { - keyFile = loadPrivateKey(this.privateKey); - } catch (IOException e) { - throw new PulsarClientException.AuthenticationException("Unable to read private key: " + e.getMessage()); - } - - // request an access token using client credentials - ClientCredentialsExchangeRequest req = ClientCredentialsExchangeRequest.builder() - .clientId(keyFile.getClientId()) - .clientSecret(keyFile.getClientSecret()) - .audience(this.audience) - .scope(this.scope) - .build(); - TokenResult tr; - if (!initialized) { - initialize(); - } - try { - tr = this.exchanger.exchangeClientCredentials(req); - } catch (TokenExchangeException | IOException e) { - throw new PulsarClientException.AuthenticationException("Unable to obtain an access token: " - + e.getMessage()); - } - - return tr; - } - - @Override - public void close() throws Exception { - if (exchanger != null) { - exchanger.close(); - } - } - /** * Constructs a {@link ClientCredentialsFlow} from configuration parameters. + * * @param params * @return */ @@ -125,16 +81,24 @@ public static ClientCredentialsFlow fromParameters(Map params) { // These are optional parameters, so we only perform a get String scope = params.get(CONFIG_PARAM_SCOPE); String audience = params.get(CONFIG_PARAM_AUDIENCE); + Duration connectTimeout = parseParameterDuration(params, CONFIG_PARAM_CONNECT_TIMEOUT); + Duration readTimeout = parseParameterDuration(params, CONFIG_PARAM_READ_TIMEOUT); + String trustCertsFilePath = params.get(CONFIG_PARAM_TRUST_CERTS_FILE_PATH); + return ClientCredentialsFlow.builder() .issuerUrl(issuerUrl) .audience(audience) .privateKey(privateKeyUrl) .scope(scope) + .connectTimeout(connectTimeout) + .readTimeout(readTimeout) + .trustCertsFilePath(trustCertsFilePath) .build(); } /** * Loads the private key from the given URL. + * * @param privateKeyURL * @return * @throws IOException @@ -162,4 +126,52 @@ private static KeyFile loadPrivateKey(String privateKeyURL) throws IOException { throw new IOException("Invalid privateKey format", e); } } + + @Override + public void initialize() throws PulsarClientException { + super.initialize(); + assert this.metadata != null; + + URL tokenUrl = this.metadata.getTokenEndpoint(); + this.exchanger = new TokenClient(tokenUrl, httpClient); + initialized = true; + } + + public TokenResult authenticate() throws PulsarClientException { + // read the private key from storage + KeyFile keyFile; + try { + keyFile = loadPrivateKey(this.privateKey); + } catch (IOException e) { + throw new PulsarClientException.AuthenticationException("Unable to read private key: " + e.getMessage()); + } + + // request an access token using client credentials + ClientCredentialsExchangeRequest req = ClientCredentialsExchangeRequest.builder() + .clientId(keyFile.getClientId()) + .clientSecret(keyFile.getClientSecret()) + .audience(this.audience) + .scope(this.scope) + .build(); + TokenResult tr; + if (!initialized) { + initialize(); + } + try { + tr = this.exchanger.exchangeClientCredentials(req); + } catch (TokenExchangeException | IOException e) { + throw new PulsarClientException.AuthenticationException("Unable to obtain an access token: " + + e.getMessage()); + } + + return tr; + } + + @Override + public void close() throws Exception { + super.close(); + if (exchanger != null) { + exchanger.close(); + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java index 125a880086297..6cc9f8e41b5e4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java @@ -18,16 +18,25 @@ */ package org.apache.pulsar.client.impl.auth.oauth2; +import io.netty.handler.ssl.SslContextBuilder; +import java.io.File; import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; +import java.time.Duration; +import java.time.format.DateTimeParseException; import java.util.Map; +import javax.net.ssl.SSLException; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver; import org.apache.pulsar.client.impl.auth.oauth2.protocol.Metadata; import org.apache.pulsar.client.impl.auth.oauth2.protocol.MetadataResolver; +import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; /** * An abstract OAuth 2.0 authorization flow. @@ -35,14 +44,60 @@ @Slf4j abstract class FlowBase implements Flow { + public static final String CONFIG_PARAM_CONNECT_TIMEOUT = "connectTimeout"; + public static final String CONFIG_PARAM_READ_TIMEOUT = "readTimeout"; + public static final String CONFIG_PARAM_TRUST_CERTS_FILE_PATH = "trustCertsFilePath"; + + protected static final Duration DEFAULT_CONNECT_TIMEOUT = Duration.ofSeconds(10); + protected static final Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(30); + private static final long serialVersionUID = 1L; protected final URL issuerUrl; + protected final AsyncHttpClient httpClient; protected transient Metadata metadata; - protected FlowBase(URL issuerUrl) { + protected FlowBase(URL issuerUrl, Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) { this.issuerUrl = issuerUrl; + this.httpClient = defaultHttpClient(readTimeout, connectTimeout, trustCertsFilePath); + } + + private AsyncHttpClient defaultHttpClient(Duration readTimeout, Duration connectTimeout, + String trustCertsFilePath) { + DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + confBuilder.setCookieStore(null); + confBuilder.setUseProxyProperties(true); + confBuilder.setFollowRedirect(true); + confBuilder.setConnectTimeout( + getParameterDurationToMillis(CONFIG_PARAM_CONNECT_TIMEOUT, connectTimeout, + DEFAULT_CONNECT_TIMEOUT)); + confBuilder.setReadTimeout( + getParameterDurationToMillis(CONFIG_PARAM_READ_TIMEOUT, readTimeout, DEFAULT_READ_TIMEOUT)); + confBuilder.setUserAgent(String.format("Pulsar-Java-v%s", PulsarVersion.getVersion())); + if (StringUtils.isNotBlank(trustCertsFilePath)) { + try { + confBuilder.setSslContext(SslContextBuilder.forClient() + .trustManager(new File(trustCertsFilePath)) + .build()); + } catch (SSLException e) { + log.error("Could not set " + CONFIG_PARAM_TRUST_CERTS_FILE_PATH, e); + } + } + return new DefaultAsyncHttpClient(confBuilder.build()); + } + + private int getParameterDurationToMillis(String name, Duration value, Duration defaultValue) { + Duration duration; + if (value == null) { + log.info("Configuration for [{}] is using the default value: [{}]", name, defaultValue); + duration = defaultValue; + } else { + log.info("Configuration for [{}] is: [{}]", name, value); + duration = value; + } + + return (int) duration.toMillis(); } public void initialize() throws PulsarClientException { @@ -55,7 +110,7 @@ public void initialize() throws PulsarClientException { } protected MetadataResolver createMetadataResolver() { - return DefaultMetadataResolver.fromIssuerUrl(issuerUrl); + return DefaultMetadataResolver.fromIssuerUrl(issuerUrl, httpClient); } static String parseParameterString(Map params, String name) { @@ -77,4 +132,21 @@ static URL parseParameterUrl(Map params, String name) { throw new IllegalArgumentException("Malformed configuration parameter: " + name); } } + + static Duration parseParameterDuration(Map params, String name) { + String value = params.get(name); + if (StringUtils.isNotBlank(value)) { + try { + return Duration.parse(value); + } catch (DateTimeParseException e) { + throw new IllegalArgumentException("Malformed configuration parameter: " + name, e); + } + } + return null; + } + + @Override + public void close() throws Exception { + httpClient.close(); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java index be636145cb24b..19d0c1acadd15 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java @@ -19,88 +19,50 @@ package org.apache.pulsar.client.impl.auth.oauth2.protocol; import com.fasterxml.jackson.databind.ObjectReader; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpHeaderValues; import java.io.IOException; import java.io.InputStream; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; -import java.net.URLConnection; -import java.time.Duration; +import java.util.concurrent.ExecutionException; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.Response; /** * Resolves OAuth 2.0 authorization server metadata as described in RFC 8414. */ public class DefaultMetadataResolver implements MetadataResolver { - protected static final int DEFAULT_CONNECT_TIMEOUT_IN_SECONDS = 10; - protected static final int DEFAULT_READ_TIMEOUT_IN_SECONDS = 30; - private final URL metadataUrl; private final ObjectReader objectReader; - private Duration connectTimeout; - private Duration readTimeout; + private final AsyncHttpClient httpClient; - public DefaultMetadataResolver(URL metadataUrl) { + public DefaultMetadataResolver(URL metadataUrl, AsyncHttpClient httpClient) { this.metadataUrl = metadataUrl; this.objectReader = ObjectMapperFactory.getMapper().reader().forType(Metadata.class); - // set a default timeout to ensure that this doesn't block - this.connectTimeout = Duration.ofSeconds(DEFAULT_CONNECT_TIMEOUT_IN_SECONDS); - this.readTimeout = Duration.ofSeconds(DEFAULT_READ_TIMEOUT_IN_SECONDS); - } - - public DefaultMetadataResolver withConnectTimeout(Duration connectTimeout) { - this.connectTimeout = connectTimeout; - return this; - } - - public DefaultMetadataResolver withReadTimeout(Duration readTimeout) { - this.readTimeout = readTimeout; - return this; - } - - /** - * Resolves the authorization metadata. - * @return metadata - * @throws IOException if the metadata could not be resolved. - */ - public Metadata resolve() throws IOException { - try { - URLConnection c = this.metadataUrl.openConnection(); - if (connectTimeout != null) { - c.setConnectTimeout((int) connectTimeout.toMillis()); - } - if (readTimeout != null) { - c.setReadTimeout((int) readTimeout.toMillis()); - } - c.setRequestProperty("Accept", "application/json"); - - Metadata metadata; - try (InputStream inputStream = c.getInputStream()) { - metadata = this.objectReader.readValue(inputStream); - } - return metadata; - - } catch (IOException e) { - throw new IOException("Cannot obtain authorization metadata from " + metadataUrl.toString(), e); - } + this.httpClient = httpClient; } /** * Gets a well-known metadata URL for the given OAuth issuer URL. + * * @param issuerUrl The authorization server's issuer identifier * @return a resolver */ - public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl) { - return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl)); + public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl, AsyncHttpClient httpClient) { + return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl), httpClient); } /** * Gets a well-known metadata URL for the given OAuth issuer URL. - * @see - * OAuth Discovery: Obtaining Authorization Server Metadata + * * @param issuerUrl The authorization server's issuer identifier * @return a URL + * @see + * OAuth Discovery: Obtaining Authorization Server Metadata */ public static URL getWellKnownMetadataUrl(URL issuerUrl) { try { @@ -109,4 +71,33 @@ public static URL getWellKnownMetadataUrl(URL issuerUrl) { throw new IllegalArgumentException(e); } } + + /** + * Resolves the authorization metadata. + * + * @return metadata + * @throws IOException if the metadata could not be resolved. + */ + public Metadata resolve() throws IOException { + + try { + Response response = httpClient.prepareGet(metadataUrl.toString()) + .addHeader(HttpHeaderNames.ACCEPT, HttpHeaderValues.APPLICATION_JSON) + .execute() + .toCompletableFuture() + .get(); + + Metadata metadata; + try (InputStream inputStream = response.getResponseBodyAsStream()) { + metadata = this.objectReader.readValue(inputStream); + } + return metadata; + + } catch (IOException | InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new IOException("Cannot obtain authorization metadata from " + metadataUrl, e); + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java index da596ce0985e3..6eee7847535b1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java @@ -27,12 +27,8 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.asynchttpclient.AsyncHttpClient; -import org.asynchttpclient.AsyncHttpClientConfig; -import org.asynchttpclient.DefaultAsyncHttpClient; -import org.asynchttpclient.DefaultAsyncHttpClientConfig; import org.asynchttpclient.Response; /** @@ -40,30 +36,11 @@ */ public class TokenClient implements ClientCredentialsExchanger { - protected static final int DEFAULT_CONNECT_TIMEOUT_IN_SECONDS = 10; - protected static final int DEFAULT_READ_TIMEOUT_IN_SECONDS = 30; - private final URL tokenUrl; private final AsyncHttpClient httpClient; - public TokenClient(URL tokenUrl) { - this(tokenUrl, null); - } - - TokenClient(URL tokenUrl, AsyncHttpClient httpClient) { - if (httpClient == null) { - DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); - confBuilder.setCookieStore(null); - confBuilder.setUseProxyProperties(true); - confBuilder.setFollowRedirect(true); - confBuilder.setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_IN_SECONDS * 1000); - confBuilder.setReadTimeout(DEFAULT_READ_TIMEOUT_IN_SECONDS * 1000); - confBuilder.setUserAgent(String.format("Pulsar-Java-v%s", PulsarVersion.getVersion())); - AsyncHttpClientConfig config = confBuilder.build(); - this.httpClient = new DefaultAsyncHttpClient(config); - } else { - this.httpClient = httpClient; - } + public TokenClient(URL tokenUrl, AsyncHttpClient httpClient) { + this.httpClient = httpClient; this.tokenUrl = tokenUrl; } @@ -74,6 +51,7 @@ public void close() throws Exception { /** * Constructing http request parameters. + * * @param req object with relevant request parameters * @return Generate the final request body from a map. */ @@ -102,6 +80,7 @@ String buildClientCredentialsBody(ClientCredentialsExchangeRequest req) { /** * Performs a token exchange using client credentials. + * * @param req the client credentials request details. * @return a token result * @throws TokenExchangeException @@ -120,24 +99,26 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re .get(); switch (res.getStatusCode()) { - case 200: - return ObjectMapperFactory.getMapper().reader().readValue(res.getResponseBodyAsBytes(), - TokenResult.class); - - case 400: // Bad request - case 401: // Unauthorized - throw new TokenExchangeException( - ObjectMapperFactory.getMapper().reader().readValue(res.getResponseBodyAsBytes(), - TokenError.class)); - - default: - throw new IOException( - "Failed to perform HTTP request. res: " + res.getStatusCode() + " " + res.getStatusText()); + case 200: + return ObjectMapperFactory.getMapper().reader().readValue(res.getResponseBodyAsBytes(), + TokenResult.class); + + case 400: // Bad request + case 401: // Unauthorized + throw new TokenExchangeException( + ObjectMapperFactory.getMapper().reader().readValue(res.getResponseBodyAsBytes(), + TokenError.class)); + + default: + throw new IOException( + "Failed to perform HTTP request. res: " + res.getStatusCode() + " " + res.getStatusText()); } - } catch (InterruptedException | ExecutionException e1) { + if (e1 instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } throw new IOException(e1); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java new file mode 100644 index 0000000000000..602aafa7b6c91 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.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.client.impl.auth.oauth2; + +import static org.testng.Assert.assertTrue; +import java.io.IOException; +import java.net.URL; +import java.time.Duration; +import org.apache.pulsar.client.api.Authentication; +import org.testng.annotations.Test; + +public class AuthenticationFactoryOAuth2Test { + + @Test + public void testBuilder() throws IOException { + URL issuerUrl = new URL("http://localhost"); + URL credentialsUrl = new URL("http://localhost"); + String audience = "audience"; + String scope = "scope"; + Duration connectTimeout = Duration.parse("PT11S"); + Duration readTimeout = Duration.ofSeconds(31); + String trustCertsFilePath = null; + try (Authentication authentication = + AuthenticationFactoryOAuth2.clientCredentialsBuilder().issuerUrl(issuerUrl) + .credentialsUrl(credentialsUrl).audience(audience).scope(scope) + .connectTimeout(connectTimeout).readTimeout(readTimeout) + .trustCertsFilePath(trustCertsFilePath).build()) { + assertTrue(authentication instanceof AuthenticationOAuth2); + } + } + + @Test + public void testClientCredentials() throws IOException { + URL issuerUrl = new URL("http://localhost"); + URL credentialsUrl = new URL("http://localhost"); + String audience = "audience"; + try (Authentication authentication = + AuthenticationFactoryOAuth2.clientCredentials(issuerUrl, credentialsUrl, audience)) { + assertTrue(authentication instanceof AuthenticationOAuth2); + } + } + +} From b89905a41b84e8ff68dc39760f7aa4fc02559c81 Mon Sep 17 00:00:00 2001 From: Hideaki Oguni <22386882+izumo27@users.noreply.github.com> Date: Fri, 6 Feb 2026 21:49:59 +0900 Subject: [PATCH 36/53] [improve][client] Make authorization server metadata path configurable in AuthenticationOAuth2 (#25052) Co-authored-by: hoguni (cherry picked from commit 3cb7a7b44c667c9777af2279cc6e7a6829c93a09) (cherry picked from commit 705a99d1109b62e244f0a3ab17f8af00ed530f3d) --- .../oauth2/AuthenticationFactoryOAuth2.java | 25 +++++++ ...thenticationOAuth2StandardAuthzServer.java | 70 +++++++++++++++++++ .../auth/oauth2/ClientCredentialsFlow.java | 7 +- .../client/impl/auth/oauth2/FlowBase.java | 8 ++- .../protocol/DefaultMetadataResolver.java | 33 +++++++-- .../AuthenticationFactoryOAuth2Test.java | 17 ++++- ...ticationOAuth2StandardAuthzServerTest.java | 50 +++++++++++++ .../auth/oauth2/AuthenticationOAuth2Test.java | 43 +++++++++++- 8 files changed, 243 insertions(+), 10 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServer.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServerTest.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java index 033d5308a2a96..7c89c6cde6d0c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java @@ -22,6 +22,7 @@ import java.time.Clock; import java.time.Duration; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver; /** * Factory class that allows to create {@link Authentication} instances @@ -69,6 +70,17 @@ public static ClientCredentialsBuilder clientCredentialsBuilder() { return new ClientCredentialsBuilder(); } + /** + * A builder to create an authentication with client credentials using standard OAuth 2.0 metadata path + * as defined in RFC 8414 ("/.well-known/oauth-authorization-server"). + * + * @return the builder pre-configured to use standard OAuth 2.0 metadata path + */ + public static ClientCredentialsBuilder clientCredentialsWithStandardAuthzServerBuilder() { + return new ClientCredentialsBuilder() + .wellKnownMetadataPath(DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH); + } + public static class ClientCredentialsBuilder { private URL issuerUrl; @@ -78,6 +90,7 @@ public static class ClientCredentialsBuilder { private Duration connectTimeout; private Duration readTimeout; private String trustCertsFilePath; + private String wellKnownMetadataPath; private ClientCredentialsBuilder() { } @@ -163,6 +176,17 @@ public ClientCredentialsBuilder trustCertsFilePath(String trustCertsFilePath) { return this; } + /** + * Optional well-known metadata path. + * + * @param wellKnownMetadataPath the well-known metadata path (must start with "/.well-known/") + * @return the builder + */ + public ClientCredentialsBuilder wellKnownMetadataPath(String wellKnownMetadataPath) { + this.wellKnownMetadataPath = wellKnownMetadataPath; + return this; + } + /** * Authenticate with client credentials. * @@ -177,6 +201,7 @@ public Authentication build() { .connectTimeout(connectTimeout) .readTimeout(readTimeout) .trustCertsFilePath(trustCertsFilePath) + .wellKnownMetadataPath(wellKnownMetadataPath) .build(); return new AuthenticationOAuth2(flow, Clock.systemDefaultZone()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServer.java new file mode 100644 index 0000000000000..c61d6d7b09747 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServer.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.impl.auth.oauth2; + +import java.io.IOException; +import java.time.Clock; +import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.impl.AuthenticationUtil; +import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver; + +/** + * Pulsar client authentication provider based on OAuth 2.0 using RFC 8414 standard metadata path. + * This class is identical to {@link AuthenticationOAuth2} but it always uses the standard + * "/.well-known/oauth-authorization-server" metadata path as defined in RFC 8414. + */ +public class AuthenticationOAuth2StandardAuthzServer extends AuthenticationOAuth2 { + + private static final long serialVersionUID = 1L; + + public AuthenticationOAuth2StandardAuthzServer() { + super(); + } + + AuthenticationOAuth2StandardAuthzServer(Flow flow, Clock clock) { + super(flow, clock); + } + + @Override + public void configure(String encodedAuthParamString) { + if (StringUtils.isBlank(encodedAuthParamString)) { + throw new IllegalArgumentException("No authentication parameters were provided"); + } + Map params; + try { + params = AuthenticationUtil.configureFromJsonString(encodedAuthParamString); + } catch (IOException e) { + throw new IllegalArgumentException("Malformed authentication parameters", e); + } + + // Always set the OAuth 2.0 standard metadata path + params.put(FlowBase.CONFIG_PARAM_WELL_KNOWN_METADATA_PATH, + DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH); + + String type = params.getOrDefault(CONFIG_PARAM_TYPE, TYPE_CLIENT_CREDENTIALS); + switch(type) { + case TYPE_CLIENT_CREDENTIALS: + this.flow = ClientCredentialsFlow.fromParameters(params); + break; + default: + throw new IllegalArgumentException("Unsupported authentication type: " + type); + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java index 7f64c0b18ac73..fe7beb47ed21f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/ClientCredentialsFlow.java @@ -62,8 +62,9 @@ class ClientCredentialsFlow extends FlowBase { @Builder public ClientCredentialsFlow(URL issuerUrl, String audience, String privateKey, String scope, - Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) { - super(issuerUrl, connectTimeout, readTimeout, trustCertsFilePath); + Duration connectTimeout, Duration readTimeout, String trustCertsFilePath, + String wellKnownMetadataPath) { + super(issuerUrl, connectTimeout, readTimeout, trustCertsFilePath, wellKnownMetadataPath); this.audience = audience; this.privateKey = privateKey; this.scope = scope; @@ -84,6 +85,7 @@ public static ClientCredentialsFlow fromParameters(Map params) { Duration connectTimeout = parseParameterDuration(params, CONFIG_PARAM_CONNECT_TIMEOUT); Duration readTimeout = parseParameterDuration(params, CONFIG_PARAM_READ_TIMEOUT); String trustCertsFilePath = params.get(CONFIG_PARAM_TRUST_CERTS_FILE_PATH); + String wellKnownMetadataPath = params.get(CONFIG_PARAM_WELL_KNOWN_METADATA_PATH); return ClientCredentialsFlow.builder() .issuerUrl(issuerUrl) @@ -93,6 +95,7 @@ public static ClientCredentialsFlow fromParameters(Map params) { .connectTimeout(connectTimeout) .readTimeout(readTimeout) .trustCertsFilePath(trustCertsFilePath) + .wellKnownMetadataPath(wellKnownMetadataPath) .build(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java index 6cc9f8e41b5e4..8a90712d7ead0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java @@ -47,6 +47,7 @@ abstract class FlowBase implements Flow { public static final String CONFIG_PARAM_CONNECT_TIMEOUT = "connectTimeout"; public static final String CONFIG_PARAM_READ_TIMEOUT = "readTimeout"; public static final String CONFIG_PARAM_TRUST_CERTS_FILE_PATH = "trustCertsFilePath"; + public static final String CONFIG_PARAM_WELL_KNOWN_METADATA_PATH = "wellKnownMetadataPath"; protected static final Duration DEFAULT_CONNECT_TIMEOUT = Duration.ofSeconds(10); protected static final Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(30); @@ -55,12 +56,15 @@ abstract class FlowBase implements Flow { protected final URL issuerUrl; protected final AsyncHttpClient httpClient; + protected final String wellKnownMetadataPath; protected transient Metadata metadata; - protected FlowBase(URL issuerUrl, Duration connectTimeout, Duration readTimeout, String trustCertsFilePath) { + protected FlowBase(URL issuerUrl, Duration connectTimeout, Duration readTimeout, String trustCertsFilePath, + String wellKnownMetadataPath) { this.issuerUrl = issuerUrl; this.httpClient = defaultHttpClient(readTimeout, connectTimeout, trustCertsFilePath); + this.wellKnownMetadataPath = wellKnownMetadataPath; } private AsyncHttpClient defaultHttpClient(Duration readTimeout, Duration connectTimeout, @@ -110,7 +114,7 @@ public void initialize() throws PulsarClientException { } protected MetadataResolver createMetadataResolver() { - return DefaultMetadataResolver.fromIssuerUrl(issuerUrl, httpClient); + return DefaultMetadataResolver.fromIssuerUrl(issuerUrl, httpClient, wellKnownMetadataPath); } static String parseParameterString(Map params, String name) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java index 19d0c1acadd15..e43117bb7df2a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/DefaultMetadataResolver.java @@ -36,6 +36,13 @@ */ public class DefaultMetadataResolver implements MetadataResolver { + private static final String WELL_KNOWN_PREFIX = "/.well-known/"; + private static final String DEFAULT_WELL_KNOWN_METADATA_PATH = WELL_KNOWN_PREFIX + "openid-configuration"; + /** + * The OAuth 2.0 Authorization Server Metadata path as defined in RFC 8414. + */ + public static final String OAUTH_WELL_KNOWN_METADATA_PATH = WELL_KNOWN_PREFIX + "oauth-authorization-server"; + private final URL metadataUrl; private final ObjectReader objectReader; private final AsyncHttpClient httpClient; @@ -50,23 +57,41 @@ public DefaultMetadataResolver(URL metadataUrl, AsyncHttpClient httpClient) { * Gets a well-known metadata URL for the given OAuth issuer URL. * * @param issuerUrl The authorization server's issuer identifier + * @param httpClient The HTTP client + * @param wellKnownMetadataPath The well-known metadata path (must start with "/.well-known/") * @return a resolver */ - public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl, AsyncHttpClient httpClient) { - return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl), httpClient); + public static DefaultMetadataResolver fromIssuerUrl(URL issuerUrl, AsyncHttpClient httpClient, + String wellKnownMetadataPath) { + return new DefaultMetadataResolver(getWellKnownMetadataUrl(issuerUrl, wellKnownMetadataPath), httpClient); } /** * Gets a well-known metadata URL for the given OAuth issuer URL. * * @param issuerUrl The authorization server's issuer identifier + * @param wellKnownMetadataPath The well-known metadata path (must start with "/.well-known/") * @return a URL * @see * OAuth Discovery: Obtaining Authorization Server Metadata */ - public static URL getWellKnownMetadataUrl(URL issuerUrl) { + public static URL getWellKnownMetadataUrl(URL issuerUrl, String wellKnownMetadataPath) { try { - return URI.create(issuerUrl.toExternalForm() + "/.well-known/openid-configuration").normalize().toURL(); + if (wellKnownMetadataPath == null || wellKnownMetadataPath.isEmpty()) { + return URI.create(issuerUrl.toExternalForm() + DEFAULT_WELL_KNOWN_METADATA_PATH).normalize().toURL(); + } + if (wellKnownMetadataPath.startsWith(WELL_KNOWN_PREFIX)) { + String issuerUrlString = issuerUrl.toExternalForm(); + // For OAuth2, insert well-known path before the issuer URL path + URL url = new URL(issuerUrlString); + String path = url.getPath(); + String basePath = issuerUrlString.substring(0, + issuerUrlString.length() - (path.isEmpty() ? 0 : path.length())); + return URI.create(basePath + wellKnownMetadataPath + path).normalize().toURL(); + } else { + throw new IllegalArgumentException("Metadata path must start with '" + WELL_KNOWN_PREFIX + + "', but was: " + wellKnownMetadataPath); + } } catch (MalformedURLException e) { throw new IllegalArgumentException(e); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java index 602aafa7b6c91..f76fee6e10dfa 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2Test.java @@ -36,11 +36,26 @@ public void testBuilder() throws IOException { Duration connectTimeout = Duration.parse("PT11S"); Duration readTimeout = Duration.ofSeconds(31); String trustCertsFilePath = null; + String wellKnownMetadataPath = "/.well-known/custom-path"; try (Authentication authentication = AuthenticationFactoryOAuth2.clientCredentialsBuilder().issuerUrl(issuerUrl) .credentialsUrl(credentialsUrl).audience(audience).scope(scope) .connectTimeout(connectTimeout).readTimeout(readTimeout) - .trustCertsFilePath(trustCertsFilePath).build()) { + .trustCertsFilePath(trustCertsFilePath) + .wellKnownMetadataPath(wellKnownMetadataPath).build()) { + assertTrue(authentication instanceof AuthenticationOAuth2); + } + } + + @Test + public void testStandardAuthzServerBuilder() throws IOException { + URL issuerUrl = new URL("http://localhost"); + URL credentialsUrl = new URL("http://localhost"); + String audience = "audience"; + String scope = "scope"; + try (Authentication authentication = + AuthenticationFactoryOAuth2.clientCredentialsWithStandardAuthzServerBuilder().issuerUrl(issuerUrl) + .credentialsUrl(credentialsUrl).audience(audience).scope(scope).build()) { assertTrue(authentication instanceof AuthenticationOAuth2); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServerTest.java new file mode 100644 index 0000000000000..e1403b3aa4ba5 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2StandardAuthzServerTest.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.client.impl.auth.oauth2; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.Map; +import org.apache.pulsar.client.impl.auth.oauth2.protocol.DefaultMetadataResolver; +import org.testng.annotations.Test; + +public class AuthenticationOAuth2StandardAuthzServerTest { + + @Test + public void testConfigureWithOAuth2MetadataPath() throws Exception { + Map params = new HashMap<>(); + params.put("type", "client_credentials"); + params.put("privateKey", "data:base64,e30="); + params.put("issuerUrl", "http://localhost"); + params.put("audience", "test-audience"); + ObjectMapper mapper = new ObjectMapper(); + String authParams = mapper.writeValueAsString(params); + AuthenticationOAuth2StandardAuthzServer auth = new AuthenticationOAuth2StandardAuthzServer(); + auth.configure(authParams); + assertTrue(auth.flow instanceof ClientCredentialsFlow); + ClientCredentialsFlow flow = (ClientCredentialsFlow) auth.flow; + Field wellKnownMetadataPathField = FlowBase.class.getDeclaredField("wellKnownMetadataPath"); + wellKnownMetadataPathField.setAccessible(true); + String wellKnownMetadataPath = (String) wellKnownMetadataPathField.get(flow); + assertEquals(wellKnownMetadataPath, DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH); + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java index aef69be74e120..d430d8f0e4066 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java @@ -85,6 +85,7 @@ public void testConfigure() throws Exception { params.put("issuerUrl", "http://localhost"); params.put("audience", "http://localhost"); params.put("scope", "http://localhost"); + params.put("wellKnownMetadataPath", "/.well-known/custom-path"); ObjectMapper mapper = new ObjectMapper(); String authParams = mapper.writeValueAsString(params); this.auth.configure(authParams); @@ -132,8 +133,48 @@ public void testGetAuthData() throws Exception { @Test public void testMetadataResolver() throws MalformedURLException { - URL url = DefaultMetadataResolver.getWellKnownMetadataUrl(URI.create("http://localhost/path/oauth").toURL()); + URL url = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + null); assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", url.toString()); + + // custom wellKnownMetadataPath with full well-known prefix + URL customUrl = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + "/.well-known/custom-path"); + assertEquals("http://localhost/.well-known/custom-path/path/oauth", customUrl.toString()); + + // null wellKnownMetadataPath (should use default) + URL customUrl2 = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + null); + assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", customUrl2.toString()); + + // empty wellKnownMetadataPath (should use default) + URL customUrl3 = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + ""); + assertEquals("http://localhost/path/oauth/.well-known/openid-configuration", customUrl3.toString()); + + // using RFC8414 OAuth2 metadata path + URL oauthUrl = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH); + assertEquals("http://localhost/.well-known/oauth-authorization-server/path/oauth", oauthUrl.toString()); + + // test with issuer URL without path + URL oauthUrlNoPath = DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost").toURL(), + DefaultMetadataResolver.OAUTH_WELL_KNOWN_METADATA_PATH); + assertEquals("http://localhost/.well-known/oauth-authorization-server", oauthUrlNoPath.toString()); + } + + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = ".*Metadata path must start with.*") + public void testMetadataResolverWithInvalidPath() throws MalformedURLException { + DefaultMetadataResolver.getWellKnownMetadataUrl( + URI.create("http://localhost/path/oauth").toURL(), + "/custom-path"); } @Test From 464cedcfe51311c6e68a0eea325848010b39a075 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 6 Feb 2026 23:02:58 +0800 Subject: [PATCH 37/53] [improve][broker] Add idle timeout support for http (#25224) (cherry picked from commit 63220eadcfc11f43d4537327a3593603ee5be697) (cherry picked from commit 144e064c145613f697933f5e959770b1ee3f81d5) --- conf/broker.conf | 3 +++ conf/proxy.conf | 6 ++++++ conf/standalone.conf | 5 ++++- .../pulsar/broker/ServiceConfiguration.java | 6 ++++++ .../org/apache/pulsar/broker/web/WebService.java | 6 +++++- .../pulsar/proxy/server/AdminProxyHandler.java | 2 ++ .../pulsar/proxy/server/ProxyConfiguration.java | 15 +++++++++++++++ .../org/apache/pulsar/proxy/server/WebServer.java | 6 +++++- 8 files changed, 46 insertions(+), 3 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index d751d423ba42d..c0f11a919314e 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1042,6 +1042,9 @@ httpServerThreadPoolQueueSize=8192 # Capacity for accept queue in the HTTP server httpServerAcceptQueueSize=8192 +# Idle timeout for HTTP server connections in milliseconds +httpServerIdleTimeout=30000 + # Maximum number of inbound http connections. (0 to disable limiting) maxHttpServerConnections=2048 diff --git a/conf/proxy.conf b/conf/proxy.conf index 10e5cdfd00d6b..0cfb554cf809a 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -315,6 +315,9 @@ httpServerThreadPoolQueueSize=8192 # Capacity for accept queue in the HTTP server httpServerAcceptQueueSize=8192 +#Idle timeout for HTTP server connections in milliseconds +httpServerIdleTimeout=30000 + # Maximum number of inbound http connections. (0 to disable limiting) maxHttpServerConnections=2048 @@ -326,6 +329,9 @@ maxConcurrentHttpRequests=1024 # denial of service attacks. httpMaxRequestHeaderSize = 8192 +# The idle timeout value for HTTP proxy is in millisecond +httpProxyIdleTimeout=30000 + ## Configure the datasource of basic authenticate, supports the file and Base64 format. # file: # basicAuthConf=/path/my/.htpasswd diff --git a/conf/standalone.conf b/conf/standalone.conf index abd4c60836958..e655b6a61ab8b 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1438,4 +1438,7 @@ topicCompactionRetainNullKey=false # If value is "org.apache.pulsar.compaction.EventTimeCompactionServiceFactory", # will create topic compaction service based on message eventTime. # By default compaction service is based on message publishing order. -compactionServiceFactoryClassName=org.apache.pulsar.compaction.PulsarCompactionServiceFactory \ No newline at end of file +compactionServiceFactoryClassName=org.apache.pulsar.compaction.PulsarCompactionServiceFactory + +# Idle timeout for HTTP server connections in milliseconds +httpServerIdleTimeout=30000 \ No newline at end of file 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 9f20b6a30402a..55c4a109f70c8 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 @@ -347,6 +347,12 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private int httpServerAcceptQueueSize = 8192; + @FieldContext( + category = CATEGORY_HTTP, + doc = "Idle timeout for HTTP server connections in milliseconds." + ) + private int httpServerIdleTimeout = 30 * 1000; + @FieldContext(category = CATEGORY_SERVER, doc = "Maximum number of inbound http connections. " + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; 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 96e8a516a6cad..7fa954948a8af 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 @@ -134,6 +134,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); + httpConfig.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { List connectionFactories = new ArrayList<>(); @@ -191,7 +192,10 @@ public WebService(PulsarService pulsar) throws PulsarServerException { } // Limit number of concurrent HTTP connections to avoid getting out of file descriptors - connectors.forEach(c -> c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize())); + connectors.forEach(c -> { + c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize()); + c.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); + }); server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); filterInitializer = new FilterInitializer(pulsar); 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 090f2f4a384b5..e4a38d7a0be2c 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 @@ -126,6 +126,8 @@ protected void customizeHttpClient(HttpClient httpClient) { protocolHandlers.put(new RedirectProtocolHandler(httpClient)); } + httpClient.setIdleTimeout(config.getHttpProxyIdleTimeout()); + setTimeout(config.getHttpProxyTimeout()); } 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 c76ad4bffe7ec..e1c5754669f4b 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 @@ -806,6 +806,14 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private int httpProxyTimeout = 5 * 60 * 1000; + @FieldContext( + minValue = 0, + category = CATEGORY_HTTP, + doc = "Http proxy idle timeout.\n\n" + + "The idle timeout value for HTTP proxy is in millisecond." + ) + private int httpProxyIdleTimeout = 30 * 1000; + @FieldContext( minValue = 1, category = CATEGORY_HTTP, @@ -830,6 +838,13 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private int httpServerAcceptQueueSize = 8192; + @FieldContext( + minValue = 0, + category = CATEGORY_HTTP, + doc = "Idle timeout for HTTP server connections in milliseconds." + ) + private int httpServerIdleTimeout = 30 * 1000; + @FieldContext(category = CATEGORY_SERVER, doc = "Maximum number of inbound http connections. " + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index 3c472135bdfb0..1d0d03d1dc73f 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 @@ -114,6 +114,7 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication } httpConfig.setOutputBufferSize(config.getHttpOutputBufferSize()); httpConfig.setRequestHeaderSize(config.getHttpMaxRequestHeaderSize()); + httpConfig.setIdleTimeout(config.getHttpServerIdleTimeout()); HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (config.getWebServicePort().isPresent()) { @@ -167,7 +168,10 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication } // Limit number of concurrent HTTP connections to avoid getting out of file descriptors - connectors.stream().forEach(c -> c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize())); + connectors.stream().forEach(c -> { + c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize()); + c.setIdleTimeout(config.getHttpServerIdleTimeout()); + }); server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); filterInitializer = new FilterInitializer(config, authenticationService); From 863703150e5f6d53065d720ebd2bfab11fddaf4a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Dec 2025 10:06:46 +0200 Subject: [PATCH 38/53] [fix][sec] Upgrade jose4j to 0.9.6 to address CVE-2024-29371 (#25095) (cherry picked from commit 3fb52c5d7494d4020956b7a26db5e9e721c7d555) (cherry picked from commit b38fe721500470c74aaf4a3615cb05746c272b6e) From dd6efa3ca62d66c41523bb1ee749ed295ecd4bcf Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 6 Feb 2026 17:32:34 +0200 Subject: [PATCH 39/53] [fix][build][branch-4.0] Fix LICENSE.bin.txt (cherry picked from commit f06f3eee542ca9e9cf230f3353df18ec8d623659) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 408c26dbdcf4b..c8d6bda8ba7ff 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -540,7 +540,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-2.21.0-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-2.21.0-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.37.0.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar From c2502752e53718f6414999a39f22a0e11f651af1 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 9 Feb 2026 16:03:25 +0800 Subject: [PATCH 40/53] [fix][broker] Fix incomplete futures in topic property update/delete methods (#25228) (cherry picked from commit c2ae180a8f0eebdbbc45dd4b8f54ae1a7015d77d) (cherry picked from commit ab05ca2d893a7bfd75cbebfb9d499ff4a059ba7f) --- .../admin/impl/PersistentTopicsBase.java | 18 +++++++++--- .../pulsar/broker/admin/AdminApi2Test.java | 29 +++++++++++++++++++ 2 files changed, 43 insertions(+), 4 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 0ccd075e7c272..0a683f4b8729d 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 @@ -660,8 +660,10 @@ private CompletableFuture internalUpdateNonPartitionedTopicProperties(Map< pulsar().getBrokerService().getTopicIfExists(topicName.toString()) .thenAccept(opt -> { if (!opt.isPresent()) { - throw new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString())); + future.completeExceptionally( + new WebApplicationException(getTopicNotFoundErrorMessage(topicName.toString()), + Status.NOT_FOUND)); + return; } ManagedLedger managedLedger = ((PersistentTopic) opt.get()).getManagedLedger(); managedLedger.asyncSetProperties(properties, new AsyncCallbacks.UpdatePropertiesCallback() { @@ -681,6 +683,9 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) future.completeExceptionally(exception); } }, null); + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; }); return future; } @@ -717,8 +722,10 @@ private CompletableFuture internalRemoveNonPartitionedTopicProperties(Stri pulsar().getBrokerService().getTopicIfExists(topicName.toString()) .thenAccept(opt -> { if (!opt.isPresent()) { - throw new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString())); + future.completeExceptionally( + new WebApplicationException(getTopicNotFoundErrorMessage(topicName.toString()), + Status.NOT_FOUND)); + return; } ManagedLedger managedLedger = ((PersistentTopic) opt.get()).getManagedLedger(); managedLedger.asyncDeleteProperty(key, new AsyncCallbacks.UpdatePropertiesCallback() { @@ -733,6 +740,9 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) future.completeExceptionally(exception); } }, null); + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; }); return future; } 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 1b5317f000fa5..7879dcefb3b79 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 @@ -1201,6 +1201,35 @@ public void testUpdateNonPartitionedTopicProperties() throws Exception { Assert.assertEquals(properties.get("key2"), "value2"); } + @Test + public void testUpdatePropertiesOnNonExistentTopic() throws Exception { + final String namespace = newUniqueName(defaultTenant + "/ns2"); + final String topicName = "persistent://" + namespace + "/testUpdatePropertiesOnNonExistentTopic"; + admin.namespaces().createNamespace(namespace, 20); + + // Test updateProperties on non-existent topic should return 404 Not Found + Map topicProperties = new HashMap<>(); + topicProperties.put("key1", "value1"); + try { + admin.topics().updateProperties(topicName, topicProperties); + fail("Should have thrown an exception for non-existent topic"); + } catch (Exception e) { + Assert.expectThrows(PulsarAdminException.NotFoundException.class, () -> { + throw e; + }); + } + + // Test removeProperties on non-existent topic should return 404 Not Found + try { + admin.topics().removeProperties(topicName, "key1"); + fail("Should have thrown an exception for non-existent topic"); + } catch (PulsarAdminException.NotFoundException e) { + Assert.expectThrows(PulsarAdminException.NotFoundException.class, () -> { + throw e; + }); + } + } + @Test public void testNonPersistentTopics() throws Exception { final String namespace = newUniqueName(defaultTenant + "/ns2"); From 6d22539d96c7352f679547402a43261b3a8fe4b6 Mon Sep 17 00:00:00 2001 From: sinan liu Date: Mon, 9 Feb 2026 17:06:17 +0800 Subject: [PATCH 41/53] [fix][test] Fix Mockito stubbing race in TopicListServiceTest (#25227) (cherry picked from commit c93dd7ad8705fa6ef9019ae796892ad2d2177b61) (cherry picked from commit 38a126b3e69c899d4a66267e3662e98fb2d3c39b) --- .../broker/service/TopicListServiceTest.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java index 5caa771289114..ec566a4734cb1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicListServiceTest.java @@ -336,6 +336,20 @@ public void testCommandWatchUpdate() { @Test public void testCommandWatchUpdateRetries() { + AtomicInteger failureCount = new AtomicInteger(0); + // Set up the stubbing before starting async work to avoid races with Mockito stubbing state. + doAnswer(invocationOnMock -> { + List newTopicsArg = invocationOnMock.getArgument(1); + if (!newTopicsArg.isEmpty() && failureCount.incrementAndGet() < 3) { + Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); + Function> permitAcquireErrorHandler = + invocationOnMock.getArgument(4); + return permitAcquireErrorHandler.apply(failure); + } else { + return CompletableFuture.completedFuture(null); + } + }).when(pulsarCommandSender).sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); + topicListService.handleWatchTopicList( NamespaceName.get("tenant/ns"), 13, @@ -349,18 +363,6 @@ public void testCommandWatchUpdateRetries() { List newTopics = Collections.singletonList("persistent://tenant/ns/topic2"); String hash = TopicList.calculateHash(ListUtils.union(topics, newTopics)); - AtomicInteger failureCount = new AtomicInteger(0); - doAnswer(invocationOnMock -> { - List newTopicsArg = invocationOnMock.getArgument(1); - if (!newTopicsArg.isEmpty() && failureCount.incrementAndGet() < 3) { - Throwable failure = new AsyncSemaphore.PermitAcquireTimeoutException("Acquire timed out"); - Function> permitAcquireErrorHandler = - invocationOnMock.getArgument(4); - return permitAcquireErrorHandler.apply(failure); - } else { - return CompletableFuture.completedFuture(null); - } - }).when(pulsarCommandSender).sendWatchTopicListUpdate(anyLong(), any(), any(), anyString(), any()); notificationConsumer.accept( new Notification(NotificationType.Created, "/managed-ledgers/tenant/ns/persistent/topic2")); notificationConsumer.accept( From a213b7c1056dc1cc7a4721277e677988036aad81 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 9 Feb 2026 18:20:37 +0800 Subject: [PATCH 42/53] [improve][broker] Give the detail error msg when authenticate failed with AuthenticationException (#25221) (cherry picked from commit 0a0ce6d012412f003b04f148548f6350fdcfb58c) (cherry picked from commit 2a46c70ba66c26776c0edfb9d9257ecea30a31d0) --- .../broker/web/AuthenticationFilter.java | 9 +- .../broker/web/AuthenticationFilterTest.java | 107 ++++++++++++++++++ 2 files changed, 114 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/AuthenticationFilterTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/AuthenticationFilter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/AuthenticationFilter.java index 3b85d9b03e4e6..1a36c4405545b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/AuthenticationFilter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/AuthenticationFilter.java @@ -60,10 +60,15 @@ public void doFilter( try { doFilter = authenticationService.authenticateHttpRequest(httpRequest, httpResponse); } catch (Exception e) { - httpResponse.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); if (e instanceof AuthenticationException) { - LOG.warn("[{}] Failed to authenticate HTTP request: {}", request.getRemoteAddr(), e.getMessage()); + String msg = e.getMessage(); + if (msg == null) { + msg = "Authentication required"; + } + httpResponse.sendError(HttpServletResponse.SC_UNAUTHORIZED, msg); + LOG.warn("[{}] Failed to authenticate HTTP request: {}", request.getRemoteAddr(), msg); } else { + httpResponse.sendError(HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); LOG.error("[{}] Error performing authentication for HTTP", request.getRemoteAddr(), e); } return; diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/AuthenticationFilterTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/AuthenticationFilterTest.java new file mode 100644 index 0000000000000..744f4a1d18975 --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/AuthenticationFilterTest.java @@ -0,0 +1,107 @@ +/* + * 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.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import javax.naming.AuthenticationException; +import javax.servlet.FilterChain; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.testng.annotations.Test; + +public class AuthenticationFilterTest { + + @Test + public void testDoFilterWithAuthenticationException() throws Exception { + AuthenticationService authenticationService = mock(AuthenticationService.class); + AuthenticationFilter filter = new AuthenticationFilter(authenticationService); + + HttpServletRequest request = mock(HttpServletRequest.class); + HttpServletResponse response = mock(HttpServletResponse.class); + FilterChain chain = mock(FilterChain.class); + + String errorMsg = "Specific authentication error"; + doThrow(new AuthenticationException(errorMsg)) + .when(authenticationService) + .authenticateHttpRequest(any(HttpServletRequest.class), any(HttpServletResponse.class)); + + filter.doFilter(request, response, chain); + + verify(response).sendError(HttpServletResponse.SC_UNAUTHORIZED, errorMsg); + } + + @Test + public void testDoFilterWithGenericException() throws Exception { + AuthenticationService authenticationService = mock(AuthenticationService.class); + AuthenticationFilter filter = new AuthenticationFilter(authenticationService); + + HttpServletRequest request = mock(HttpServletRequest.class); + HttpServletResponse response = mock(HttpServletResponse.class); + FilterChain chain = mock(FilterChain.class); + + String errorMsg = "Some internal error"; + doThrow(new RuntimeException(errorMsg)) + .when(authenticationService) + .authenticateHttpRequest(any(HttpServletRequest.class), any(HttpServletResponse.class)); + + filter.doFilter(request, response, chain); + + verify(response).sendError(HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); + } + + @Test + public void testDoFilterWithNullMessageGenericException() throws Exception { + AuthenticationService authenticationService = mock(AuthenticationService.class); + AuthenticationFilter filter = new AuthenticationFilter(authenticationService); + + HttpServletRequest request = mock(HttpServletRequest.class); + HttpServletResponse response = mock(HttpServletResponse.class); + FilterChain chain = mock(FilterChain.class); + + doThrow(new RuntimeException()) + .when(authenticationService) + .authenticateHttpRequest(any(HttpServletRequest.class), any(HttpServletResponse.class)); + + filter.doFilter(request, response, chain); + + verify(response).sendError(HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); + } + + @Test + public void testDoFilterWithNullMessageAuthenticationException() throws Exception { + AuthenticationService authenticationService = mock(AuthenticationService.class); + AuthenticationFilter filter = new AuthenticationFilter(authenticationService); + + HttpServletRequest request = mock(HttpServletRequest.class); + HttpServletResponse response = mock(HttpServletResponse.class); + FilterChain chain = mock(FilterChain.class); + + doThrow(new AuthenticationException(null)) + .when(authenticationService) + .authenticateHttpRequest(any(HttpServletRequest.class), any(HttpServletResponse.class)); + + filter.doFilter(request, response, chain); + + verify(response).sendError(HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); + } +} From 02b00b295a74df3f8c046b87d937a03df439e0a3 Mon Sep 17 00:00:00 2001 From: cai minjian <905767378@qq.com> Date: Mon, 9 Feb 2026 18:24:40 +0800 Subject: [PATCH 43/53] [fix][client] Send all chunkMessageIds to broker for redelivery (#25229) (cherry picked from commit 0a0ce6d012412f003b04f148548f6350fdcfb58c) (cherry picked from commit f49c7b288a1d0f62cafe2bf80b25aa53c097dc2a) --- .../impl/MessageChunkingSharedTest.java | 33 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 1 - .../client/impl/NegativeAcksTracker.java | 22 ++++++++++++- 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingSharedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingSharedTest.java index 3d24d3746d66a..203715ca7dbcf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingSharedTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingSharedTest.java @@ -193,6 +193,39 @@ public void testInterleavedChunks() throws Exception { assertEquals(receivedUuidList1, Arrays.asList("A-0", "B-0", "B-1", "A-1")); } + // Issue #25220 + @Test + public void testNegativeAckChunkedMessage() throws Exception { + final String topic = "persistent://my-property/my-ns/test-negative-acknowledge-with-chunk"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .negativeAckRedeliveryDelay(1, TimeUnit.SECONDS) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(false) + .enableChunking(true) + .chunkMaxMessageSize(1024) // 1KB max - forces chunking for larger messages + .create(); + String longMessage = "X".repeat(10 * 1024); + producer.sendAsync(longMessage); + producer.flush(); + + // negative ack the first message + consumer.negativeAcknowledge(consumer.receive()); + + // now 2s has passed, the first message should be redelivered 1s later. + Message msg1 = consumer.receive(2, TimeUnit.SECONDS); + assertNotNull(msg1); + } + private Producer createProducer(String topic) throws PulsarClientException { return pulsarClient.newProducer(Schema.STRING) .topic(topic) 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 c1e8df4d35c19..50c5cccf85527 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 @@ -1486,7 +1486,6 @@ void messageReceived(CommandMessage cmdMessage, ByteBuf headersAndPayload, Clien // and return undecrypted payload if (isMessageUndecryptable || (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch())) { - // right now, chunked messages are only supported by non-shared subscription if (isChunkedMessage) { uncompressedPayload = processMessageChunk(uncompressedPayload, msgMetadata, msgId, messageId, cnx); if (uncompressedPayload == null) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index 273880569c307..c6983d047a277 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -47,6 +47,7 @@ class NegativeAcksTracker implements Closeable { // different timestamp, there will be multiple entries in the map // RB Tree -> LongOpenHashMap -> Roaring64Bitmap private Long2ObjectSortedMap> nackedMessages = null; + private final Long2ObjectMap> nackedMessageIds = new Long2ObjectOpenHashMap<>(); private final ConsumerBase consumer; private final Timer timer; @@ -89,7 +90,17 @@ private void triggerRedelivery(Timeout t) { long ledgerId = ledgerEntry.getLongKey(); Roaring64Bitmap entrySet = ledgerEntry.getValue(); entrySet.forEach(entryId -> { - MessageId msgId = new MessageIdImpl(ledgerId, entryId, DUMMY_PARTITION_INDEX); + MessageId msgId = null; + Long2ObjectMap entryMap = nackedMessageIds.get(ledgerId); + if (entryMap != null) { + msgId = entryMap.remove(entryId); + if (entryMap.isEmpty()) { + nackedMessageIds.remove(ledgerId); + } + } + if (msgId == null) { + msgId = new MessageIdImpl(ledgerId, entryId, DUMMY_PARTITION_INDEX); + } addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); messagesToRedeliver.add(msgId); }); @@ -143,6 +154,12 @@ static long trimLowerBit(long timestamp, int bits) { } private synchronized void add(MessageId messageId, int redeliveryCount) { + if (messageId instanceof ChunkMessageIdImpl) { + MessageIdAdv msgId = (MessageIdAdv) messageId; + nackedMessageIds.computeIfAbsent(msgId.getLedgerId(), k -> new Long2ObjectOpenHashMap<>()) + .put(msgId.getEntryId(), messageId); + } + if (nackedMessages == null) { nackedMessages = new Long2ObjectAVLTreeMap<>(); } @@ -201,5 +218,8 @@ public synchronized void close() { nackedMessages.clear(); nackedMessages = null; } + if (nackedMessageIds != null) { + nackedMessageIds.clear(); + } } } From d53e2994fec9a6c3d9affd90548ff098f88c08a2 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 9 Feb 2026 19:15:58 +0800 Subject: [PATCH 44/53] [fix][broker] Fix transactionMetadataFuture completeExceptionally with null value (#25231) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 张浩 (cherry picked from commit 0e5d4246f3abc7235fd2547899742ecd85d4d593) (cherry picked from commit 42283f4a37e14e19913a9d88d9392a6a21fc11c6) --- .../org/apache/pulsar/broker/admin/impl/TransactionsBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 55767136f8151..e3764c896862a 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 @@ -260,7 +260,7 @@ private void getTransactionMetadata(TxnMeta txnMeta, FutureUtil.waitForAll(producedPartitionsFutures).whenComplete((x, t) -> { if (t != null) { - transactionMetadataFuture.completeExceptionally(e); + transactionMetadataFuture.completeExceptionally(t); return; } From c8cf38e2261b69a3178922713ae11a4e27cf07d4 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Wed, 11 Feb 2026 11:08:36 +0530 Subject: [PATCH 45/53] [fix] Licsense issue with opentelemetry detector jar --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c8d6bda8ba7ff..408c26dbdcf4b 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -540,7 +540,7 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-2.21.0-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-2.21.0-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.37.0.jar - - com.google.cloud.opentelemetry-detector-resources-support-0.36.0.jar + - com.google.cloud.opentelemetry-detector-resources-support-0.33.0.jar - io.opentelemetry.contrib-opentelemetry-gcp-resources-1.48.0-alpha.jar * Spotify completable-futures - com.spotify-completable-futures-0.3.6.jar From c777e3a0bf04df91a4053976d5474da5674358f7 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Wed, 11 Feb 2026 12:14:02 +0530 Subject: [PATCH 46/53] [fix] correcting mismatch conflicts --- .../main/java/org/apache/pulsar/broker/web/WebService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 7fa954948a8af..4e6d9d9b5c57f 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 @@ -134,7 +134,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); - httpConfig.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); + httpConfig.setIdleTimeout(config.getHttpServerIdleTimeout()); HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { List connectionFactories = new ArrayList<>(); @@ -194,7 +194,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { // Limit number of concurrent HTTP connections to avoid getting out of file descriptors connectors.forEach(c -> { c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize()); - c.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); + c.setIdleTimeout(config.getHttpServerIdleTimeout()); }); server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); From c83f48aff439cbe7a42b75659523d42b8062ccca Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Wed, 11 Feb 2026 13:02:39 +0530 Subject: [PATCH 47/53] [fix] correcting webservice CI check error --- .../main/java/org/apache/pulsar/broker/web/WebService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 4e6d9d9b5c57f..7fa954948a8af 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 @@ -134,7 +134,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); - httpConfig.setIdleTimeout(config.getHttpServerIdleTimeout()); + httpConfig.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { List connectionFactories = new ArrayList<>(); @@ -194,7 +194,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { // Limit number of concurrent HTTP connections to avoid getting out of file descriptors connectors.forEach(c -> { c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize()); - c.setIdleTimeout(config.getHttpServerIdleTimeout()); + c.setIdleTimeout(pulsar.getConfig().getHttpServerIdleTimeout()); }); server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); From 3a2bf1e4fb11fb4eb106c8b8c394d4729f89ce12 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Wed, 11 Feb 2026 13:09:02 +0530 Subject: [PATCH 48/53] fix shell license correction --- distribution/shell/src/assemble/LICENSE.bin.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 2e4dd89ca159e..0eb8d8b846119 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -391,9 +391,9 @@ The Apache Software License, Version 2.0 - opentelemetry-context-1.56.0.jar * BookKeeper - - bookkeeper-common-allocator-4.17.1.0.0.4.jar - - cpu-affinity-4.17.1.0.0.4.jar - - circe-checksum-4.17.1.0.0.4.jar + - bookkeeper-common-allocator-4.17.1.0.0.5.jar + - cpu-affinity-4.17.1.0.0.5.jar + - circe-checksum-4.17.1.0.0.5.jar * AirCompressor - aircompressor-0.27.jar * AsyncHttpClient From a0e4b6a4c70295a8b3c907f2558773bf2ed42763 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Thu, 12 Feb 2026 22:11:59 +0530 Subject: [PATCH 49/53] fix lombok getter issue --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 4 ++++ .../org/apache/pulsar/proxy/server/ProxyConfiguration.java | 4 ++++ 2 files changed, 8 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 55c4a109f70c8..3874a18af27d2 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 @@ -4038,6 +4038,10 @@ public int getTopicOrderedExecutorThreadNum() { ? numWorkerThreadsForNonPersistentTopic : topicOrderedExecutorThreadNum; } + public int getHttpServerIdleTimeout() { + return httpServerIdleTimeout; + } + public Map lookupProperties() { final var map = new HashMap(); properties.forEach((key, value) -> { 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 e1c5754669f4b..f5bd8a9c10796 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 @@ -1076,4 +1076,8 @@ public int getMetadataStoreCacheExpirySeconds() { public boolean isMetadataStoreAllowReadOnlyOperations() { return zooKeeperAllowReadOnlyOperations || metadataStoreAllowReadOnlyOperations; } + + public int getHttpServerIdleTimeout() { + return httpServerIdleTimeout; + } } From 3efce1b251e227129287e2063b93315a215c918b Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Sat, 14 Feb 2026 19:05:09 +0530 Subject: [PATCH 50/53] fix trying to test if offline build fix the lombok error --- pom.xml | 20 ++++++++++---------- pulsar-bom/pom.xml | 20 ++++++++++---------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index aa2c86254c9e3..9a1d3ba46b5b9 100644 --- a/pom.xml +++ b/pom.xml @@ -3261,16 +3261,16 @@ flexible messaging model and an intuitive client API. - - 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-bom/pom.xml b/pulsar-bom/pom.xml index 2cd6f81b5c40a..b73e658029a9c 100644 --- a/pulsar-bom/pom.xml +++ b/pulsar-bom/pom.xml @@ -130,16 +130,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 22b67e40007a79a25fdaecd99902db07ebda5117 Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Sat, 14 Feb 2026 21:55:33 +0530 Subject: [PATCH 51/53] fix for readLedgerHandler() CI test failure --- .../org/apache/bookkeeper/client/PulsarMockBookKeeper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 8848369f30277..54b8853fb0d71 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -172,7 +172,8 @@ public void asyncCreateLedger(int ensSize, int qSize, DigestType digestType, byt } @Override - public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx) { + public void asyncOpenLedger(long lId, DigestType digestType, byte[] passwd, OpenCallback cb, Object ctx, + boolean keepUpdateMetadata) { getProgrammedFailure().thenComposeAsync((res) -> { PulsarMockLedgerHandle lh = ledgers.get(lId); if (lh == null) { From 7bb3348aa19f5600666f20051a96204dfb315e0f Mon Sep 17 00:00:00 2001 From: coderzc Date: Fri, 13 Feb 2026 10:49:13 +0800 Subject: [PATCH 52/53] Revert "[improve][meta] PIP-453: Improve the metadata store threading model (#25187)" This reverts commit c5f7271499e8879353139b9b1ee021a8dda111ad. (cherry picked from commit a6aab863b4a86b5dcb9be21045f1333f1c4501f2) --- conf/broker.conf | 2 - conf/standalone.conf | 3 - pip/pip-453.md | 26 ++- ...IsolatedBookieEnsemblePlacementPolicy.java | 3 - .../pulsar/broker/ServiceConfiguration.java | 6 - ...atedBookieEnsemblePlacementPolicyTest.java | 34 ++-- .../apache/pulsar/broker/PulsarService.java | 2 - .../pulsar/broker/PulsarServiceTest.java | 65 ------ .../OpenTelemetryMetadataStoreStatsTest.java | 12 ++ .../metadata/api/MetadataStoreConfig.java | 3 - .../cache/impl/MetadataCacheImpl.java | 185 +++++++----------- .../metadata/impl/AbstractMetadataStore.java | 77 +++----- .../metadata/impl/EtcdMetadataStore.java | 19 +- .../impl/LocalMemoryMetadataStore.java | 2 +- .../metadata/impl/RocksdbMetadataStore.java | 2 +- .../pulsar/metadata/impl/ZKMetadataStore.java | 19 +- .../AbstractBatchedMetadataStore.java | 39 ++-- .../metadata/impl/oxia/OxiaMetadataStore.java | 4 +- .../impl/stats/BatchMetadataStoreStats.java | 43 +++- .../impl/MetadataStoreFactoryImplTest.java | 2 +- 20 files changed, 232 insertions(+), 316 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index c0f11a919314e..6da34883f7743 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -920,8 +920,6 @@ metadataStoreBatchingMaxOperations=1000 # Maximum size of a batch metadataStoreBatchingMaxSizeKb=128 -# The number of threads used for serializing and deserializing data to and from the metadata store -metadataStoreSerDesThreads=1 ### --- Authentication --- ### diff --git a/conf/standalone.conf b/conf/standalone.conf index e655b6a61ab8b..5a3989fd18728 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -430,9 +430,6 @@ metadataStoreBatchingMaxOperations=1000 # Maximum size of a batch metadataStoreBatchingMaxSizeKb=128 -# The number of threads used for serializing and deserializing data to and from the metadata store -metadataStoreSerDesThreads=1 - ### --- TLS --- ### # Deprecated - Use webServicePortTls and brokerServicePortTls instead tlsEnabled=false diff --git a/pip/pip-453.md b/pip/pip-453.md index f9109798ba778..a42736b9dda71 100644 --- a/pip/pip-453.md +++ b/pip/pip-453.md @@ -40,9 +40,8 @@ Additionally, some code paths execute the compute intensive tasks in the metadat # High Level Design -Create 4 sets of threads: +Create 3 set of threads: - `-event`: the original metadata store thread, which is now only responsible to handle notifications. This executor won't be a `ScheduledExecutorService` anymore. -- `-scheduler`: a single thread, which is used to schedule tasks like flushing and retrying failed operations. - `-batch-flusher`: a single thread, which is used to schedule the flushing task at a fixed rate. It won't be created if `metadataStoreBatchingEnabled` is false. - `-worker`: a fixed thread pool shared by all `MetadataCache` instances to execute compute intensive tasks like serialization and deserialization. The same path will be handled by the same thread to keep the processing order on the same path. @@ -54,6 +53,25 @@ The only concern is that introducing a new thread to execute callbacks allows wa metadataStore.get(path).thenApply(__ -> metadataStore.get(otherPath).join());; ``` +Other tasks like the retry on failure is executed in JVM's common `ForkJoinPool` by `CompletableFuture` APIs. For example: + +```diff +--- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java ++++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +@@ -245,9 +245,8 @@ public class ZKMetadataStore extends AbstractBatchedMetadataStore + countsByType, totalSize, opsForLog); + + // Retry with the individual operations +- executor.schedule(() -> { +- ops.forEach(o -> batchOperation(Collections.singletonList(o))); +- }, 100, TimeUnit.MILLISECONDS); ++ CompletableFuture.delayedExecutor(100, TimeUnit.MILLISECONDS).execute(() -> ++ ops.forEach(o -> batchOperation(Collections.singletonList(o)))); + } else { + MetadataStoreException e = getException(code, path); + ops.forEach(o -> o.getFuture().completeExceptionally(e)); +``` + # Detailed Design ## Public-facing Changes @@ -67,11 +85,9 @@ Add a configurations to specify the number of worker threads for `MetadataCache` category = CATEGORY_SERVER, doc = "The number of threads uses for serializing and deserializing data to and from the metadata store" ) - private int metadataStoreSerDesThreads = 1; + private int metadataStoreSerDesThreads = Runtime.getRuntime().availableProcessors(); ``` -Use 1 as the default value since the serialization and deserialization tasks are not frequent. This separated thread pool is mainly added to avoid blocking the metadata store callback thread. - ### Metrics The `pulsar_batch_metadata_store_executor_queue_size` metric will be removed because the `-batch-flusher` thread won't execute other tasks except for flushing. 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 4ef1c594be444..878bbc4d654a7 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 @@ -28,7 +28,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; @@ -58,8 +57,6 @@ public class IsolatedBookieEnsemblePlacementPolicy extends RackawareEnsemblePlac // the secondary group. private ImmutablePair, Set> defaultIsolationGroups; - @Getter - @VisibleForTesting private MetadataCache bookieMappingCache; private static final String PULSAR_SYSTEM_TOPIC_ISOLATION_GROUP = "*"; 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 3874a18af27d2..e1f1483adf294 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 @@ -490,12 +490,6 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private boolean metadataStoreAllowReadOnlyOperations; - @FieldContext( - category = CATEGORY_SERVER, - doc = "The number of threads used for serializing and deserializing data to and from the metadata store" - ) - private int metadataStoreSerDesThreads = 1; - @Deprecated @FieldContext( category = CATEGORY_SERVER, 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 936b04386ff7b..68f92ab416dc2 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 @@ -22,15 +22,12 @@ 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.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.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -291,7 +288,8 @@ public void testBasic() throws Exception { secondaryBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); bookieMapping.put("group2", secondaryBookieGroup); - updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), + Optional.empty()).join(); ensemble = isolationPolicy.newEnsemble(2, 2, 2, Collections.emptyMap(), null).getResult(); @@ -342,7 +340,8 @@ public void testNoBookieInfo() throws Exception { + "\": {\"rack\": \"rack0\", \"hostname\": \"bookie3.example.com\"}, \"" + BOOKIE4 + "\": {\"rack\": \"rack2\", \"hostname\": \"bookie4.example.com\"}}}"; - updateBookieInfo(isolationPolicy, data.getBytes(StandardCharsets.UTF_8)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, data.getBytes(StandardCharsets.UTF_8), + Optional.empty()).join(); List ensemble = isolationPolicy.newEnsemble(2, 2, 2, Collections.emptyMap(), new HashSet<>()).getResult(); @@ -400,7 +399,8 @@ public void testBookieInfoChange() throws Exception { bookieMapping.put("group1", mainBookieGroup); bookieMapping.put("group2", secondaryBookieGroup); - updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), + Optional.empty()).join(); ensemble = isolationPolicy.newEnsemble(3, 3, 3, Collections.emptyMap(), new HashSet<>()).getResult(); @@ -784,7 +784,8 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup2, group2); bookieMapping.put(isolationGroup3, group3); - updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), + Optional.empty()).join(); groups.setLeft(Sets.newHashSet(isolationGroup1, isolationGroup3)); groups.setRight(Sets.newHashSet("")); @@ -807,7 +808,8 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup1, group1); bookieMapping.put(isolationGroup2, group2); - updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), + Optional.empty()).join(); groups.setLeft(Sets.newHashSet(isolationGroup1)); groups.setRight(Sets.newHashSet(isolationGroup2)); @@ -829,24 +831,12 @@ public void testGetExcludedBookiesWithIsolationGroups() throws Exception { bookieMapping.put(isolationGroup1, group1); bookieMapping.put(isolationGroup2, group2); - updateBookieInfo(isolationPolicy, jsonMapper.writeValueAsBytes(bookieMapping)); + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, jsonMapper.writeValueAsBytes(bookieMapping), + Optional.empty()).join(); groups.setLeft(Sets.newHashSet(isolationGroup1)); groups.setRight(Sets.newHashSet(isolationGroup2)); blacklist = isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); assertTrue(blacklist.isEmpty()); } - - // The policy gets the bookie info asynchronously before each query or update, when putting the bookie info into - // the metadata store, the cache needs some time to receive the notification and update accordingly. - private void updateBookieInfo(IsolatedBookieEnsemblePlacementPolicy isolationPolicy, byte[] bookieInfo) { - final var cache = isolationPolicy.getBookieMappingCache(); - assertNotNull(cache); // the policy must have been initialized - - final var key = BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH; - final var previousBookieInfo = cache.getIfCached(key); - store.put(key, bookieInfo, Optional.empty()).join(); - Awaitility.await().atMost(Duration.ofSeconds(1)).untilAsserted(() -> - assertNotEquals(cache.getIfCached(key), previousBookieInfo)); - } } 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 53907fe8fb495..fd9c74b3e82bd 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 @@ -429,7 +429,6 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro .synchronizer(synchronizer) .openTelemetry(openTelemetry) .nodeSizeStats(new DefaultMetadataNodeSizeStats()) - .numSerDesThreads(config.getMetadataStoreSerDesThreads()) .build()); } @@ -1300,7 +1299,6 @@ public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchro .metadataStoreName(MetadataStoreConfig.METADATA_STORE) .openTelemetry(openTelemetry) .nodeSizeStats(new DefaultMetadataNodeSizeStats()) - .numSerDesThreads(config.getMetadataStoreSerDesThreads()) .build()); } 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 6195e9cdae593..6c04889d8f1ba 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 @@ -25,15 +25,10 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertSame; -import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -43,10 +38,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.metadata.api.MetadataCacheConfig; -import org.apache.pulsar.metadata.api.MetadataSerde; -import org.apache.pulsar.metadata.api.MetadataStore; -import org.apache.pulsar.metadata.api.Stat; import org.testng.annotations.AfterMethod; import org.testng.annotations.Test; @@ -348,60 +339,4 @@ public void testShutdownViaAdminApi() throws Exception { assertTrue(e instanceof PulsarClientException.TimeoutException); } } - - @Test - public void testMetadataSerDesThreads() throws Exception { - final var numSerDesThreads = 5; - final var config = new ServiceConfiguration(); - config.setMetadataStoreSerDesThreads(numSerDesThreads); - config.setClusterName("test"); - config.setMetadataStoreUrl("memory:local"); - config.setConfigurationMetadataStoreUrl("memory:local"); - - @Cleanup final var pulsar = new PulsarService(config); - pulsar.start(); - - BiConsumer verifier = (store, prefix) -> { - final var serDes = new CustomMetadataSerDes(); - final var cache = store.getMetadataCache(prefix, serDes, MetadataCacheConfig.builder().build()); - for (int i = 0; i < 100 && serDes.threadNameToSerializedPaths.size() < numSerDesThreads; i++) { - cache.create(prefix + i, "value-" + i).join(); - final var value = cache.get(prefix + i).join(); - assertEquals(value.orElseThrow(), "value-" + i); - final var newValue = cache.readModifyUpdate(prefix + i, s -> s + "-updated").join(); - assertEquals(newValue, "value-" + i + "-updated"); - // Verify the serialization and deserialization are handled by the same thread - assertEquals(serDes.threadNameToSerializedPaths, serDes.threadNameToDeserializedPaths); - } - log.info("SerDes thread mapping: {}", serDes.threadNameToSerializedPaths); - assertEquals(serDes.threadNameToSerializedPaths.keySet().size(), numSerDesThreads); - // Verify a path cannot be handled by multiple threads - final var paths = serDes.threadNameToSerializedPaths.values().stream() - .flatMap(Set::stream).sorted().toList(); - assertEquals(paths.stream().distinct().toList(), paths); - }; - - verifier.accept(pulsar.getLocalMetadataStore(), "/test-local/"); - verifier.accept(pulsar.getConfigurationMetadataStore(), "/test-config/"); - } - - private static class CustomMetadataSerDes implements MetadataSerde { - - final Map> threadNameToSerializedPaths = new ConcurrentHashMap<>(); - final Map> threadNameToDeserializedPaths = new ConcurrentHashMap<>(); - - @Override - public byte[] serialize(String path, String value) throws IOException{ - threadNameToSerializedPaths.computeIfAbsent(Thread.currentThread().getName(), - __ -> ConcurrentHashMap.newKeySet()).add(path); - return value.getBytes(); - } - - @Override - public String deserialize(String path, byte[] data, Stat stat) throws IOException { - threadNameToDeserializedPaths.computeIfAbsent(Thread.currentThread().getName(), - __ -> ConcurrentHashMap.newKeySet()).add(path); - return new String(data); - } - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java index 390aa1e49e29d..9e8bde20b88e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.assertj.core.api.Assertions.assertThat; import io.opentelemetry.api.common.Attributes; +import java.util.concurrent.ExecutorService; import lombok.Cleanup; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.BrokerTestUtil; @@ -28,6 +29,7 @@ import org.apache.pulsar.broker.testcontext.NonClosingProxyHandler; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.impl.stats.BatchMetadataStoreStats; import org.apache.pulsar.metadata.impl.stats.MetadataStoreStats; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -51,6 +53,14 @@ protected void setup() throws Exception { var newStats = new MetadataStoreStats( localMetadataStoreName, pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); FieldUtils.writeField(localMetadataStore, "metadataStoreStats", newStats, true); + + var currentBatchedStats = (BatchMetadataStoreStats) FieldUtils.readField(localMetadataStore, + "batchMetadataStoreStats", true); + currentBatchedStats.close(); + var currentExecutor = (ExecutorService) FieldUtils.readField(currentBatchedStats, "executor", true); + var newBatchedStats = new BatchMetadataStoreStats(localMetadataStoreName, currentExecutor, + pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); + FieldUtils.writeField(localMetadataStore, "batchMetadataStoreStats", newBatchedStats, true); } @AfterMethod(alwaysRun = true) @@ -79,5 +89,7 @@ public void testMetadataStoreStats() throws Exception { var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); assertMetricLongSumValue(metrics, MetadataStoreStats.METADATA_STORE_PUT_BYTES_COUNTER_METRIC_NAME, attributes, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, BatchMetadataStoreStats.EXECUTOR_QUEUE_SIZE_METRIC_NAME, attributes, + value -> assertThat(value).isPositive()); } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java index fcde0dce8404b..ef50dc87691d0 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java @@ -104,7 +104,4 @@ public class MetadataStoreConfig { * The estimator to estimate the payload length of metadata node, which used to limit the batch size requested. */ private MetadataNodeSizeStats nodeSizeStats; - - @Builder.Default - private final int numSerDesThreads = 1; } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index ca165f0464e44..b1f0572547ca7 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -39,10 +39,10 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.pulsar.common.stats.CacheMetricsCollector; import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.metadata.api.CacheGetResult; +import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataSerde; @@ -62,26 +62,23 @@ public class MetadataCacheImpl implements MetadataCache, Consumer serde; - private final OrderedExecutor executor; - private final ScheduledExecutorService schedulerExecutor; + private final ScheduledExecutorService executor; private final MetadataCacheConfig cacheConfig; private final AsyncLoadingCache>> objCache; public MetadataCacheImpl(String cacheName, MetadataStore store, TypeReference typeRef, - MetadataCacheConfig cacheConfig, OrderedExecutor executor, - ScheduledExecutorService schedulerExecutor) { - this(cacheName, store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor, schedulerExecutor); + MetadataCacheConfig cacheConfig, ScheduledExecutorService executor) { + this(cacheName, store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig, executor); } public MetadataCacheImpl(String cacheName, MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig, - OrderedExecutor executor, ScheduledExecutorService schedulerExecutor) { - this(cacheName, store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor, schedulerExecutor); + ScheduledExecutorService executor) { + this(cacheName, store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig, executor); } public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde serde, - MetadataCacheConfig cacheConfig, OrderedExecutor executor, - ScheduledExecutorService schedulerExecutor) { + MetadataCacheConfig cacheConfig, ScheduledExecutorService executor) { this.store = store; if (store instanceof MetadataStoreExtended) { this.storeExtended = (MetadataStoreExtended) store; @@ -91,7 +88,6 @@ public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde this.serde = serde; this.cacheConfig = cacheConfig; this.executor = executor; - this.schedulerExecutor = schedulerExecutor; Caffeine cacheBuilder = Caffeine.newBuilder(); if (cacheConfig.getRefreshAfterWriteMillis() > 0) { @@ -105,9 +101,6 @@ public MetadataCacheImpl(String cacheName, MetadataStore store, MetadataSerde .buildAsync(new AsyncCacheLoader>>() { @Override public CompletableFuture>> asyncLoad(String key, Executor executor) { - if (log.isDebugEnabled()) { - log.debug("Loading key {} into metadata cache {}", key, cacheName); - } return readValueFromStore(key); } @@ -117,16 +110,12 @@ public CompletableFuture>> asyncReload( Optional> oldValue, Executor executor) { if (store instanceof AbstractMetadataStore && ((AbstractMetadataStore) store).isConnected()) { - if (log.isDebugEnabled()) { - log.debug("Reloading key {} into metadata cache {}", key, cacheName); - } - final var future = readValueFromStore(key); - future.thenAccept(val -> { + return readValueFromStore(key).thenApply(val -> { if (cacheConfig.getAsyncReloadConsumer() != null) { cacheConfig.getAsyncReloadConsumer().accept(key, val); } + return val; }); - return future; } else { // Do not try to refresh the cache item if we know that we're not connected to the // metadata store @@ -139,46 +128,22 @@ public CompletableFuture>> asyncReload( } private CompletableFuture>> readValueFromStore(String path) { - final var future = new CompletableFuture>>(); - store.get(path).thenComposeAsync(optRes -> { - // There could be multiple pending reads for the same path, for example, when a path is created, - // 1. The `accept` method will call `refresh` - // 2. The `put` method will call `refresh` after the metadata store put operation is done - // Both will call this method and the same result will be read. In this case, we only need to deserialize - // the value once. - if (!optRes.isPresent()) { - if (log.isDebugEnabled()) { - log.debug("Key {} not found in metadata store", path); - } - return FutureUtils.value(Optional.>empty()); - } - final var res = optRes.get(); - final var cachedFuture = objCache.getIfPresent(path); - if (cachedFuture != null && cachedFuture != future) { - if (log.isDebugEnabled()) { - log.debug("A new read on key {} is in progress or completed, ignore this one", path); - } - return cachedFuture; - } - try { - T obj = serde.deserialize(path, res.getValue(), res.getStat()); - if (log.isDebugEnabled()) { - log.debug("Deserialized value for key {} (version: {}): {}", path, res.getStat().getVersion(), - obj); - } - return FutureUtils.value(Optional.of(new CacheGetResult<>(obj, res.getStat()))); - } catch (Throwable t) { - return FutureUtils.exception(new ContentDeserializationException( - "Failed to deserialize payload for key '" + path + "'", t)); - } - }, executor.chooseThread(path)).whenComplete((result, e) -> { - if (e != null) { - future.completeExceptionally(e.getCause()); - } else { - future.complete(result); - } - }); - return future; + return store.get(path) + .thenCompose(optRes -> { + if (!optRes.isPresent()) { + return FutureUtils.value(Optional.empty()); + } + + try { + GetResult res = optRes.get(); + T obj = serde.deserialize(path, res.getValue(), res.getStat()); + return FutureUtils + .value(Optional.of(new CacheGetResult<>(obj, res.getStat()))); + } catch (Throwable t) { + return FutureUtils.exception(new ContentDeserializationException( + "Failed to deserialize payload for key '" + path + "'", t)); + } + }); } @Override @@ -204,9 +169,8 @@ public Optional getIfCached(String path) { @Override public CompletableFuture readModifyUpdateOrCreate(String path, Function, T> modifyFunction) { - final var executor = this.executor.chooseThread(path); return executeWithRetry(() -> objCache.get(path) - .thenComposeAsync(optEntry -> { + .thenCompose(optEntry -> { Optional currentValue; long expectedVersion; @@ -238,14 +202,13 @@ public CompletableFuture readModifyUpdateOrCreate(String path, Function { refresh(path); }).thenApply(__ -> newValueObj); - }, executor), path); + }), path); } @Override public CompletableFuture readModifyUpdate(String path, Function modifyFunction) { - final var executor = this.executor.chooseThread(path); return executeWithRetry(() -> objCache.get(path) - .thenComposeAsync(optEntry -> { + .thenCompose(optEntry -> { if (!optEntry.isPresent()) { return FutureUtils.exception(new NotFoundException("")); } @@ -268,57 +231,59 @@ public CompletableFuture readModifyUpdate(String path, Function modifyF return store.put(path, newValue, Optional.of(expectedVersion)).thenAccept(__ -> { refresh(path); }).thenApply(__ -> newValueObj); - }, executor), path); - } - - private CompletableFuture serialize(String path, T value) { - final var future = new CompletableFuture(); - executor.executeOrdered(path, () -> { - try { - future.complete(serde.serialize(path, value)); - } catch (Throwable t) { - future.completeExceptionally(t); - } - }); - return future; + }), path); } @Override public CompletableFuture create(String path, T value) { - final var future = new CompletableFuture(); - serialize(path, value).thenCompose(content -> store.put(path, content, Optional.of(-1L))) - // Make sure we have the value cached before the operation is completed - // In addition to caching the value, we need to add a watch on the path, - // so when/if it changes on any other node, we are notified and we can - // update the cache - .thenCompose(__ -> objCache.get(path)) - .whenComplete((__, ex) -> { - if (ex == null) { - future.complete(null); - } else if (ex.getCause() instanceof BadVersionException) { - // Use already exists exception to provide more self-explanatory error message - future.completeExceptionally(new AlreadyExistsException(ex.getCause())); - } else { - future.completeExceptionally(ex.getCause()); - } - }); + byte[] content; + try { + content = serde.serialize(path, value); + } catch (Throwable t) { + return FutureUtils.exception(t); + } + + CompletableFuture future = new CompletableFuture<>(); + store.put(path, content, Optional.of(-1L)) + .thenAccept(stat -> { + // Make sure we have the value cached before the operation is completed + // In addition to caching the value, we need to add a watch on the path, + // so when/if it changes on any other node, we are notified and we can + // update the cache + objCache.get(path).whenComplete((stat2, ex) -> { + if (ex == null) { + future.complete(null); + } else { + log.error("Exception while getting path {}", path, ex); + future.completeExceptionally(ex.getCause()); + } + }); + }).exceptionally(ex -> { + if (ex.getCause() instanceof BadVersionException) { + // Use already exists exception to provide more self-explanatory error message + future.completeExceptionally(new AlreadyExistsException(ex.getCause())); + } else { + future.completeExceptionally(ex.getCause()); + } + return null; + }); + return future; } @Override public CompletableFuture put(String path, T value, EnumSet options) { - return serialize(path, value).thenCompose(bytes -> { - if (storeExtended != null) { - return storeExtended.put(path, bytes, Optional.empty(), options); - } else { - return store.put(path, bytes, Optional.empty()); - } - }).thenAccept(__ -> { - if (log.isDebugEnabled()) { - log.debug("Refreshing path {} after put operation", path); - } - refresh(path); - }); + final byte[] bytes; + try { + bytes = serde.serialize(path, value); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } + if (storeExtended != null) { + return storeExtended.put(path, bytes, Optional.empty(), options).thenAccept(__ -> refresh(path)); + } else { + return store.put(path, bytes, Optional.empty()).thenAccept(__ -> refresh(path)); + } } @Override @@ -358,9 +323,6 @@ public void accept(Notification t) { switch (t.getType()) { case Created: case Modified: - if (log.isDebugEnabled()) { - log.debug("Refreshing path {} for {} notification", path, t.getType()); - } refresh(path); break; @@ -392,7 +354,8 @@ private void execute(Supplier> op, String key, CompletableF final var next = backoff.next(); log.info("Update key {} conflicts. Retrying in {} ms. Mandatory stop: {}. Elapsed time: {} ms", key, next, backoff.isMandatoryStopMade(), elapsed); - schedulerExecutor.schedule(() -> execute(op, key, result, backoff), next, TimeUnit.MILLISECONDS); + executor.schedule(() -> execute(op, key, result, backoff), next, + TimeUnit.MILLISECONDS); return null; } result.completeExceptionally(ex.getCause()); 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 d118a792e2f23..b0e4b43f70067 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 @@ -26,7 +26,6 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.MoreExecutors; import io.netty.util.concurrent.DefaultThreadFactory; import io.opentelemetry.api.OpenTelemetry; import java.time.Instant; @@ -39,17 +38,16 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; +import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.stats.CacheMetricsCollector; import org.apache.pulsar.common.util.FutureUtil; @@ -78,9 +76,7 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList> sessionListeners = new CopyOnWriteArrayList<>(); protected final String metadataStoreName; - private final OrderedExecutor serDesExecutor; - private final ExecutorService eventExecutor; - private final ScheduledExecutorService schedulerExecutor; + protected final ScheduledExecutorService executor; private final AsyncLoadingCache> childrenCache; private final AsyncLoadingCache existsCache; private final CopyOnWriteArrayList> metadataCaches = new CopyOnWriteArrayList<>(); @@ -97,21 +93,13 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co protected MetadataNodeSizeStats nodeSizeStats; - protected AbstractMetadataStore( - String metadataStoreName, OpenTelemetry openTelemetry, MetadataNodeSizeStats nodeSizeStats, - int numSerDesThreads) { + protected AbstractMetadataStore(String metadataStoreName, OpenTelemetry openTelemetry, + MetadataNodeSizeStats nodeSizeStats) { this.nodeSizeStats = nodeSizeStats == null ? new DummyMetadataNodeSizeStats() : nodeSizeStats; - final var namePrefix = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName - : getClass().getSimpleName(); - this.eventExecutor = Executors.newSingleThreadExecutor( - new DefaultThreadFactory(namePrefix + "-event")); - this.schedulerExecutor = Executors.newSingleThreadScheduledExecutor( - new DefaultThreadFactory(namePrefix + "-scheduler")); - this.serDesExecutor = OrderedExecutor.newBuilder() - .numThreads(numSerDesThreads) - .name(namePrefix + "-worker") - .build(); + this.executor = new ScheduledThreadPoolExecutor(1, + new DefaultThreadFactory( + StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName())); registerListener(this); this.childrenCache = Caffeine.newBuilder() @@ -261,8 +249,7 @@ public MetadataCache getMetadataCache(Class clazz, MetadataCacheConfig JavaType typeRef = TypeFactory.defaultInstance().constructSimpleType(clazz, null); String cacheName = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName(); MetadataCacheImpl metadataCache = - new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.serDesExecutor, - this.schedulerExecutor); + new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.executor); metadataCaches.add(metadataCache); return metadataCache; } @@ -271,8 +258,7 @@ public MetadataCache getMetadataCache(Class clazz, MetadataCacheConfig public MetadataCache getMetadataCache(TypeReference typeRef, MetadataCacheConfig cacheConfig) { String cacheName = StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName(); MetadataCacheImpl metadataCache = - new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.serDesExecutor, - this.schedulerExecutor); + new MetadataCacheImpl(cacheName, this, typeRef, cacheConfig, this.executor); metadataCaches.add(metadataCache); return metadataCache; } @@ -282,7 +268,7 @@ public MetadataCache getMetadataCache(String cacheName, MetadataSerde MetadataCacheConfig cacheConfig) { MetadataCacheImpl metadataCache = new MetadataCacheImpl<>(cacheName, this, serde, cacheConfig, - this.serDesExecutor, this.schedulerExecutor); + this.executor); metadataCaches.add(metadataCache); return metadataCache; } @@ -362,7 +348,7 @@ protected CompletableFuture receivedNotification(Notification notification }); return null; - }, eventExecutor); + }, executor); } catch (RejectedExecutionException e) { return FutureUtil.failedFuture(e); } @@ -545,7 +531,7 @@ protected void receivedSessionEvent(SessionEvent event) { // Notice listeners. try { - eventExecutor.execute(() -> { + executor.execute(() -> { sessionListeners.forEach(l -> { try { l.accept(event); @@ -570,9 +556,8 @@ protected static CompletableFuture alreadyClosedFailedFuture() { @Override public void close() throws Exception { - MoreExecutors.shutdownAndAwaitTermination(serDesExecutor, 10, TimeUnit.SECONDS); - MoreExecutors.shutdownAndAwaitTermination(schedulerExecutor, 10, TimeUnit.SECONDS); - MoreExecutors.shutdownAndAwaitTermination(eventExecutor, 10, TimeUnit.SECONDS); + executor.shutdownNow(); + executor.awaitTermination(10, TimeUnit.SECONDS); this.metadataStoreStats.close(); } @@ -589,30 +574,30 @@ public void invalidateCaches(String...paths) { } } - protected final void processEvent(Consumer eventProcessor, T event) { + /** + * Run the task in the executor thread and fail the future if the executor is shutting down. + */ + @VisibleForTesting + public void execute(Runnable task, CompletableFuture future) { try { - eventExecutor.execute(() -> eventProcessor.accept(event)); - } catch (RejectedExecutionException e) { - log.warn("Rejected processing event {}", event); + executor.execute(task); + } catch (Throwable t) { + future.completeExceptionally(t); } } - protected final void scheduleDelayedTask(long delay, TimeUnit unit, Runnable task) { - schedulerExecutor.schedule(task, delay, unit); - } - - protected final void safeExecuteCallback(Runnable task, Consumer exceptionHandler) { + /** + * Run the task in the executor thread and fail the future if the executor is shutting down. + */ + @VisibleForTesting + public void execute(Runnable task, Supplier>> futures) { try { - eventExecutor.execute(task); - } catch (Throwable t) { - exceptionHandler.accept(t); + executor.execute(task); + } catch (final Throwable t) { + futures.get().forEach(f -> f.completeExceptionally(t)); } } - protected final void safeExecuteCallback(Runnable task, CompletableFuture future) { - safeExecuteCallback(task, future::completeExceptionally); - } - protected static String parent(String path) { int idx = path.lastIndexOf('/'); if (idx <= 0) { 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 e1311fccfe034..3937fd712dc9f 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 @@ -188,7 +188,7 @@ public void close() throws Exception { @Override protected CompletableFuture existsFromStore(String path) { return kv.get(ByteSequence.from(path, StandardCharsets.UTF_8), EXISTS_GET_OPTION) - .thenApply(gr -> gr.getCount() == 1); + .thenApplyAsync(gr -> gr.getCount() == 1, executor); } @Override @@ -204,8 +204,9 @@ protected CompletableFuture storePut(String path, byte[] data, Optional super.storePut(path + stat.getVersion(), data, optExpectedVersion, options)); + .thenComposeAsync( + stat -> super.storePut(path + stat.getVersion(), data, optExpectedVersion, options), + executor); } } @@ -312,7 +313,9 @@ protected void batchOperation(List ops) { } } else { log.warn("Failed to commit: {}", cause.getMessage()); - ops.forEach(o -> o.getFuture().completeExceptionally(ex)); + executor.execute(() -> { + ops.forEach(o -> o.getFuture().completeExceptionally(ex)); + }); } return null; }); @@ -323,7 +326,7 @@ protected void batchOperation(List ops) { private void handleBatchOperationResult(TxnResponse txnResponse, List ops) { - safeExecuteCallbacks(() -> { + executor.execute(() -> { if (!txnResponse.isSucceeded()) { if (ops.size() > 1) { // Retry individually @@ -401,7 +404,7 @@ private void handleBatchOperationResult(TxnResponse txnResponse, } } } - }, ops); + }); } private synchronized CompletableFuture createLease(boolean retryOnFailure) { @@ -441,7 +444,9 @@ public void onCompleted() { if (retryOnFailure) { future.exceptionally(ex -> { log.warn("Failed to create Etcd lease. Retrying later", ex); - scheduleDelayedTask(1, TimeUnit.SECONDS, () -> createLease(true)); + executor.schedule(() -> { + createLease(true); + }, 1, TimeUnit.SECONDS); return null; }); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index 627304b2edc7d..079cb3130e054 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -79,7 +79,7 @@ private static class Value { public LocalMemoryMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); + metadataStoreConfig.getNodeSizeStats()); String name = metadataURL.substring(MEMORY_SCHEME_IDENTIFIER.length()); // Local means a private data set // update synchronizer and register sync listener 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 08e5478ffcca1..74bddda7454cb 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 @@ -210,7 +210,7 @@ static long toLong(byte[] bytes) { private RocksdbMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); + metadataStoreConfig.getNodeSizeStats()); this.metadataUrl = metadataURL; try { RocksDB.loadLibrary(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index f56d6c6941f1e..5bf7e2272f022 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -119,7 +119,7 @@ public ZKMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConf private void processSessionWatcher(WatchedEvent event) { if (sessionWatcher != null) { - processEvent(sessionWatcher::process, event); + executor.execute(() -> sessionWatcher.process(event)); } } @@ -245,8 +245,9 @@ protected void batchOperation(List ops) { countsByType, totalSize, opsForLog); // Retry with the individual operations - scheduleDelayedTask(100, TimeUnit.MILLISECONDS, - () -> ops.forEach(o -> batchOperation(Collections.singletonList(o)))); + executor.schedule(() -> { + ops.forEach(o -> batchOperation(Collections.singletonList(o))); + }, 100, TimeUnit.MILLISECONDS); } else { MetadataStoreException e = getException(code, path); ops.forEach(o -> o.getFuture().completeExceptionally(e)); @@ -255,7 +256,7 @@ protected void batchOperation(List ops) { } // Trigger all the futures in the batch - safeExecuteCallbacks(() -> { + execute(() -> { for (int i = 0; i < ops.size(); i++) { OpResult opr = results.get(i); MetadataOp op = ops.get(i); @@ -277,7 +278,7 @@ protected void batchOperation(List ops) { "Operation type not supported in multi: " + op.getType())); } } - }, ops); + }, () -> ops.stream().map(MetadataOp::getFuture).collect(Collectors.toList())); }, null); } catch (Throwable t) { ops.forEach(o -> o.getFuture().completeExceptionally(new MetadataStoreException(t))); @@ -394,7 +395,7 @@ public CompletableFuture existsFromStore(String path) { try { zkc.exists(path, null, (rc, path1, ctx, stat) -> { - safeExecuteCallback(() -> { + execute(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(true); @@ -420,7 +421,7 @@ private void internalStoreDelete(OpDelete op) { try { zkc.delete(op.getPath(), expectedVersion, (rc, path1, ctx) -> { - safeExecuteCallback(() -> { + execute(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(null); @@ -445,7 +446,7 @@ private void internalStorePut(OpPut opPut) { CreateMode createMode = getCreateMode(opPut.getOptions()); asyncCreateFullPathOptimistic(zkc, opPut.getPath(), opPut.getData(), createMode, (rc, path1, ctx, name) -> { - safeExecuteCallback(() -> { + execute(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(new Stat(name, 0, 0, 0, createMode.isEphemeral(), true)); @@ -459,7 +460,7 @@ private void internalStorePut(OpPut opPut) { }); } else { zkc.setData(opPut.getPath(), opPut.getData(), expectedVersion, (rc, path1, ctx, stat) -> { - safeExecuteCallback(() -> { + execute(() -> { Code code = Code.get(rc); if (code == Code.OK) { future.complete(getStat(path1, stat)); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index 30989a41bd167..a9319a50fec5c 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -18,20 +18,16 @@ */ package org.apache.pulsar.metadata.impl.batching; -import com.google.common.util.concurrent.MoreExecutors; -import io.netty.util.concurrent.DefaultThreadFactory; import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; -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.AtomicBoolean; import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataEventSynchronizer; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -42,7 +38,6 @@ import org.apache.pulsar.metadata.impl.stats.BatchMetadataStoreStats; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.MpscUnboundedArrayQueue; -import org.jspecify.annotations.Nullable; @Slf4j public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore { @@ -51,6 +46,8 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private final MessagePassingQueue readOps; private final MessagePassingQueue writeOps; + private final AtomicBoolean flushInProgress = new AtomicBoolean(false); + private final boolean enabled; private final int maxDelayMillis; protected final int maxOperations; @@ -58,12 +55,9 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private MetadataEventSynchronizer synchronizer; private final BatchMetadataStoreStats batchMetadataStoreStats; protected MetadataStoreBatchStrategy metadataStoreBatchStrategy; - @Nullable - private final ScheduledExecutorService flushExecutor; protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { - super(conf.getMetadataStoreName(), conf.getOpenTelemetry(), conf.getNodeSizeStats(), - conf.getNumSerDesThreads()); + super(conf.getMetadataStoreName(), conf.getOpenTelemetry(), conf.getNodeSizeStats()); this.enabled = conf.isBatchingEnabled(); this.maxDelayMillis = conf.getBatchingMaxDelayMillis(); @@ -73,22 +67,18 @@ protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { if (enabled) { readOps = new MpscUnboundedArrayQueue<>(10_000); writeOps = new MpscUnboundedArrayQueue<>(10_000); - final var name = StringUtils.isNotBlank(conf.getMetadataStoreName()) ? conf.getMetadataStoreName() - : getClass().getSimpleName(); - flushExecutor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory( - name + "-batch-flusher")); - scheduledTask = flushExecutor.scheduleAtFixedRate(this::flush, maxDelayMillis, maxDelayMillis, - TimeUnit.MILLISECONDS); + scheduledTask = + executor.scheduleAtFixedRate(this::flush, maxDelayMillis, maxDelayMillis, TimeUnit.MILLISECONDS); } else { scheduledTask = null; readOps = null; writeOps = null; - flushExecutor = null; } // update synchronizer and register sync listener updateMetadataEventSynchronizer(conf.getSynchronizer()); - this.batchMetadataStoreStats = new BatchMetadataStoreStats(metadataStoreName); + this.batchMetadataStoreStats = + new BatchMetadataStoreStats(metadataStoreName, executor, conf.getOpenTelemetry()); this.metadataStoreBatchStrategy = new DefaultMetadataStoreBatchStrategy(maxOperations, maxSize); } @@ -106,13 +96,12 @@ public void close() throws Exception { op.getFuture().completeExceptionally(ex); } scheduledTask.cancel(true); - MoreExecutors.shutdownAndAwaitTermination(flushExecutor, 10, TimeUnit.SECONDS); } super.close(); this.batchMetadataStoreStats.close(); } - private synchronized void flush() { + private void flush() { List currentBatch; if (!readOps.isEmpty()) { while (CollectionUtils.isNotEmpty(currentBatch = metadataStoreBatchStrategy.nextBatch(readOps))) { @@ -124,6 +113,8 @@ private synchronized void flush() { internalBatchOperation(currentBatch); } } + + flushInProgress.set(false); } @Override @@ -178,8 +169,8 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { internalBatchOperation(Collections.singletonList(op)); return; } - if (queue.size() > maxOperations) { - flush(); + if (queue.size() > maxOperations && flushInProgress.compareAndSet(false, true)) { + executor.execute(this::flush); } } else { internalBatchOperation(Collections.singletonList(op)); @@ -203,8 +194,4 @@ private void internalBatchOperation(List ops) { } protected abstract void batchOperation(List ops); - - protected final void safeExecuteCallbacks(Runnable runnable, List ops) { - safeExecuteCallback(runnable, t -> ops.forEach(op -> op.getFuture().completeExceptionally(t))); - } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 407a927bda4dc..d055dd7da55fb 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -61,7 +61,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { private Optional synchronizer; public OxiaMetadataStore(AsyncOxiaClient oxia, String identity) { - super("oxia-metadata", OpenTelemetry.noop(), null, 1); + super("oxia-metadata", OpenTelemetry.noop(), null); this.client = oxia; this.identity = identity; this.synchronizer = Optional.empty(); @@ -75,7 +75,7 @@ public OxiaMetadataStore( boolean enableSessionWatcher) throws Exception { super("oxia-metadata", Objects.requireNonNull(metadataStoreConfig).getOpenTelemetry(), - metadataStoreConfig.getNodeSizeStats(), metadataStoreConfig.getNumSerDesThreads()); + metadataStoreConfig.getNodeSizeStats()); var linger = metadataStoreConfig.getBatchingMaxDelayMillis(); if (!metadataStoreConfig.isBatchingEnabled()) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java index 82cc15d8aafab..9549a8df8f9f1 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java @@ -18,13 +18,23 @@ */ package org.apache.pulsar.metadata.impl.stats; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; +import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; public final class BatchMetadataStoreStats implements AutoCloseable { private static final double[] BUCKETS = new double[]{1, 5, 10, 20, 50, 100, 200, 500, 1000}; private static final String NAME = "name"; + private static final Gauge EXECUTOR_QUEUE_SIZE = Gauge + .build("pulsar_batch_metadata_store_executor_queue_size", "-") + .labelNames(NAME) + .register(); private static final Histogram OPS_WAITING = Histogram .build("pulsar_batch_metadata_store_queue_wait_time", "-") .unit("ms") @@ -44,17 +54,46 @@ public final class BatchMetadataStoreStats implements AutoCloseable { .register(); private final AtomicBoolean closed = new AtomicBoolean(false); + private final ThreadPoolExecutor executor; private final String metadataStoreName; private final Histogram.Child batchOpsWaitingChild; private final Histogram.Child batchExecuteTimeChild; private final Histogram.Child opsPerBatchChild; - public BatchMetadataStoreStats(String metadataStoreName) { + public static final String EXECUTOR_QUEUE_SIZE_METRIC_NAME = "pulsar.broker.metadata.store.executor.queue.size"; + private final ObservableLongUpDownCounter batchMetadataStoreSizeCounter; + + public BatchMetadataStoreStats(String metadataStoreName, ExecutorService executor, OpenTelemetry openTelemetry) { + if (executor instanceof ThreadPoolExecutor tx) { + this.executor = tx; + } else { + this.executor = null; + } this.metadataStoreName = metadataStoreName; + + EXECUTOR_QUEUE_SIZE.setChild(new Gauge.Child() { + @Override + public double get() { + return getQueueSize(); + } + }, metadataStoreName); + this.batchOpsWaitingChild = OPS_WAITING.labels(metadataStoreName); this.batchExecuteTimeChild = BATCH_EXECUTE_TIME.labels(metadataStoreName); this.opsPerBatchChild = OPS_PER_BATCH.labels(metadataStoreName); + + var meter = openTelemetry.getMeter("org.apache.pulsar"); + var attributes = Attributes.of(MetadataStoreStats.METADATA_STORE_NAME, metadataStoreName); + this.batchMetadataStoreSizeCounter = meter + .upDownCounterBuilder(EXECUTOR_QUEUE_SIZE_METRIC_NAME) + .setDescription("The number of batch operations in the metadata store executor queue") + .setUnit("{operation}") + .buildWithCallback(measurement -> measurement.record(getQueueSize(), attributes)); + } + + private int getQueueSize() { + return executor == null ? 0 : executor.getQueue().size(); } public void recordOpWaiting(long millis) { @@ -72,9 +111,11 @@ public void recordOpsInBatch(int ops) { @Override public void close() throws Exception { if (closed.compareAndSet(false, true)) { + EXECUTOR_QUEUE_SIZE.remove(this.metadataStoreName); OPS_WAITING.remove(this.metadataStoreName); BATCH_EXECUTE_TIME.remove(this.metadataStoreName); OPS_PER_BATCH.remove(metadataStoreName); + batchMetadataStoreSizeCounter.close(); } } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java index 0ae0b022a352d..d42b2228346b8 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java @@ -96,7 +96,7 @@ public MetadataStore create(String metadataURL, MetadataStoreConfig metadataStor public static class MyMetadataStore extends AbstractMetadataStore { protected MyMetadataStore() { - super("custom", OpenTelemetry.noop(), null, 1); + super("custom", OpenTelemetry.noop(), null); } @Override From 7717857647268ba9824084227366fa5e9a3e108b Mon Sep 17 00:00:00 2001 From: hshankar31 Date: Sun, 15 Feb 2026 23:22:24 +0530 Subject: [PATCH 53/53] revert commented dist management --- pom.xml | 20 ++++++++++---------- pulsar-bom/pom.xml | 20 ++++++++++---------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 9a1d3ba46b5b9..aa2c86254c9e3 100644 --- a/pom.xml +++ b/pom.xml @@ -3261,16 +3261,16 @@ flexible messaging model and an intuitive client API. - - - - - - - - - - + + 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-bom/pom.xml b/pulsar-bom/pom.xml index b73e658029a9c..2cd6f81b5c40a 100644 --- a/pulsar-bom/pom.xml +++ b/pulsar-bom/pom.xml @@ -130,16 +130,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/ +