From a51bbdd1480b453d72891b3d68b4afd068ec374a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 8 Apr 2024 10:34:10 +0800 Subject: [PATCH 01/55] [improve][broker] Deprecate unused enableNamespaceIsolationUpdateOnTime config (#22449) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 5 ++--- .../java/org/apache/pulsar/broker/admin/AdminApi2Test.java | 3 ++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 80dfcaf4b0b20..324a4c9a8dc01 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 @@ -1453,11 +1453,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Enable or disable exposing broker entry metadata to client.") private boolean exposingBrokerEntryMetadataToClientEnabled = false; + @Deprecated @FieldContext( category = CATEGORY_SERVER, - doc = "Enable namespaceIsolation policy update take effect ontime or not," - + " if set to ture, then the related namespaces will be unloaded after reset policy to make it " - + "take effect." + doc = "This config never takes effect and will be removed in the next release" ) private boolean enableNamespaceIsolationUpdateOnTime = false; 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 3899338870451..6bc64f4dd65d0 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 @@ -199,7 +199,6 @@ protected ServiceConfiguration getDefaultConf() { void configureDefaults(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); conf.setLoadBalancerEnabled(true); - conf.setEnableNamespaceIsolationUpdateOnTime(true); conf.setAllowOverrideEntryFilters(true); conf.setEntryFilterNames(List.of()); conf.setMaxNumPartitionsPerPartitionedTopic(0); @@ -1394,6 +1393,7 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { try { admin.lookups().lookupTopic(ns1Name + "/topic3"); + fail(); } catch (Exception e) { // expected lookup fail, because no brokers matched the policy. log.info(" 2 expected fail lookup"); @@ -1401,6 +1401,7 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { try { admin.lookups().lookupTopic(ns1Name + "/topic1"); + fail(); } catch (Exception e) { // expected lookup fail, because no brokers matched the policy. log.info(" 22 expected fail lookup"); From 57a616eaa79096af5b49db89c99cd39ccc94ec00 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 8 Apr 2024 18:22:05 +0800 Subject: [PATCH 02/55] [fix][broker] Fix consumer stops receiving messages when with large backlogs processing (#22454) --- .../mledger/impl/ManagedCursorImpl.java | 5 ++ .../mledger/impl/ManagedLedgerImpl.java | 9 +-- .../persistent/PersistentSubscription.java | 4 +- .../persistent/PersistentTopicTest.java | 56 ++++++++++++++++++- 4 files changed, 65 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 4daa06cad576a..69b130a98c869 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 @@ -990,6 +990,11 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition); } + if (isClosed()) { + callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); + return; + } + if (!hasMoreEntries()) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Still no entries available. Register for notification", ledger.getName(), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 3a12cb2ad6c74..698563ed7a1f2 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 @@ -1032,6 +1032,7 @@ public synchronized void asyncDeleteCursor(final String consumerName, final Dele + consumerName), ctx); return; } else if (!cursor.isDurable()) { + cursor.setState(ManagedCursorImpl.State.Closed); cursors.removeCursor(consumerName); deactivateCursorByName(consumerName); callback.deleteCursorComplete(ctx); @@ -3814,13 +3815,7 @@ public void removeWaitingCursor(ManagedCursor cursor) { } public void addWaitingCursor(ManagedCursorImpl cursor) { - if (cursor instanceof NonDurableCursorImpl) { - if (cursor.isActive()) { - this.waitingCursors.add(cursor); - } - } else { - this.waitingCursors.add(cursor); - } + this.waitingCursors.add(cursor); } public boolean isCursorActive(ManagedCursor cursor) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 6e8e94baeae23..dbbf92aa76dce 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -308,7 +308,6 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (dispatcher != null && dispatcher.getConsumers().isEmpty()) { deactivateCursor(); - topic.getManagedLedger().removeWaitingCursor(cursor); if (!cursor.isDurable()) { // If cursor is not durable, we need to clean up the subscription as well. No need to check for active @@ -338,11 +337,14 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (!isResetCursor) { try { topic.getManagedLedger().deleteCursor(cursor.getName()); + topic.getManagedLedger().removeWaitingCursor(cursor); } catch (InterruptedException | ManagedLedgerException e) { log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); } } }); + } else { + topic.getManagedLedger().removeWaitingCursor(cursor); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index d42b1d92007aa..c214634e6ed32 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -80,6 +80,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -113,6 +114,11 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerCursorBackloggedThreshold(10); + } + /** * Test validates that broker cleans up topic which failed to unload while bundle unloading. * @@ -681,7 +687,7 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); doAnswer((invocation) -> { - Thread.sleep(10_000); + Thread.sleep(5_000); invocation.callRealMethod(); return null; }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), @@ -708,4 +714,52 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { assertEquals(ledger.getWaitingCursorsCount(), 0); }); } + + @Test + public void testAddWaitingCursorsForNonDurable2() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors2"; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName("sub-1").subscribe().close(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + int count = 0; + while(true) { + final Message msg = consumer.receive(3, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 100); + Thread.sleep(3_000); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + while(true) { + final Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 200); + } } From b162d46906961623db09c97df3f06b7876cddb5e Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:14:25 +0800 Subject: [PATCH 03/55] [cleanup][broker] remove useless code comment (#22459) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 1650e449a3fd6..b21cd165402e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3875,9 +3875,6 @@ private void unfenceTopicToResume() { @Override public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishContext publishContext) { pendingWriteOps.incrementAndGet(); - // in order to avoid the opAddEntry retain - - // in order to promise the publish txn message orderly, we should change the transactionCompletableFuture if (isFenced) { publishContext.completed(new TopicFencedException("fenced"), -1, -1); From 3bdb30c7a6151ec97c25865fbbd0bb24613ab991 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 9 Apr 2024 14:59:13 +0800 Subject: [PATCH 04/55] [fix][build] Fix ps command (#22451) Signed-off-by: Zixuan Liu --- docker/pulsar/Dockerfile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 1ca6edb2e323c..f586a9dd4f9d7 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -97,7 +97,8 @@ RUN apk add --no-cache \ python3 \ py3-pip \ gcompat \ - ca-certificates + ca-certificates \ + procps # Install GLibc compatibility library COPY --from=glibc /root/packages /root/packages @@ -106,6 +107,9 @@ RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk COPY --from=jvm /opt/jvm /opt/jvm ENV JAVA_HOME=/opt/jvm +# The default is /pulsat/bin and cannot be written. +ENV PULSAR_PID_DIR=/pulsar/logs + # Copy Python depedencies from the other stage COPY --from=python-deps /usr/lib/python3.11/site-packages /usr/lib/python3.11/site-packages From 6de711d4008338a875c5c145e856c90dcb041f8f Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 9 Apr 2024 16:38:18 +0800 Subject: [PATCH 05/55] [improve][test] Add operation authentication test for namespace API (#22398) --- .../broker/admin/NamespaceAuthZTest.java | 882 +++++++++++++++++- 1 file changed, 875 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index ce0b925614c55..d5a0468f340c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -19,23 +19,47 @@ package org.apache.pulsar.broker.admin; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import io.jsonwebtoken.Jwts; +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; import lombok.Cleanup; import lombok.SneakyThrows; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; +import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.packages.management.core.MockedPackagesStorageProvider; +import org.apache.pulsar.packages.management.core.common.PackageMetadata; import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.UUID; @Test(groups = "broker-admin") public class NamespaceAuthZTest extends MockedPulsarStandalone { @@ -44,17 +68,27 @@ public class NamespaceAuthZTest extends MockedPulsarStandalone { private PulsarAdmin tenantManagerAdmin; + private PulsarClient pulsarClient; + + private AuthorizationService authorizationService; + + private AuthorizationService orignalAuthorizationService; + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows @BeforeClass - public void before() { + public void setup() { + getServiceConfiguration().setEnablePackagesManagement(true); + getServiceConfiguration().setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName()); + getServiceConfiguration().setDefaultNumberOfNamespaceBundles(1); + getServiceConfiguration().setForceDeleteNamespaceAllowed(true); configureTokenAuthentication(); configureDefaultAuthorization(); start(); - this.superUserAdmin =PulsarAdmin.builder() + this.superUserAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) .build(); @@ -65,12 +99,13 @@ public void before() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); + this.pulsarClient = super.getPulsarService().getClient(); } @SneakyThrows @AfterClass - public void after() { + public void cleanup() { if (superUserAdmin != null) { superUserAdmin.close(); } @@ -80,6 +115,33 @@ public void after() { close(); } + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException, PulsarAdminException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + superUserAdmin.namespaces().deleteNamespace("public/default", true); + superUserAdmin.namespaces().createNamespace("public/default"); + } + + private void setAuthorizationOperationChecker(String role, NamespaceOperation operation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + NamespaceOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any()); + } @SneakyThrows @Test @@ -160,4 +222,810 @@ public void testProperties() { } superUserAdmin.topics().delete(topic, true); } + + @Test + public void testTopics() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // test super admin + superUserAdmin.namespaces().getTopics(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().getTopics(namespace); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getTopics(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_TOPICS); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action || AuthAction.produce == action) { + subAdmin.namespaces().getTopics(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getTopics(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testBookieAffinityGroup() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // test super admin + BookieAffinityGroupData bookieAffinityGroupData = BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("aaa") + .bookkeeperAffinityGroupSecondary("bbb") + .build(); + superUserAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData); + BookieAffinityGroupData bookieAffinityGroup = superUserAdmin.namespaces().getBookieAffinityGroup(namespace); + Assert.assertEquals(bookieAffinityGroupData, bookieAffinityGroup); + superUserAdmin.namespaces().deleteBookieAffinityGroup(namespace); + bookieAffinityGroup = superUserAdmin.namespaces().getBookieAffinityGroup(namespace); + Assert.assertNull(bookieAffinityGroup); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + + @Test + public void testGetBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test super admin + superUserAdmin.namespaces().getBundles(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().getBundles(namespace); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBundles(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_BUNDLE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action || AuthAction.produce == action) { + subAdmin.namespaces().getBundles(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBundles(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnloadBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testSplitBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testDeleteBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + for (int i = 0; i < 3; i++) { + superUserAdmin.namespaces().splitNamespaceBundle(namespace, Policies.BundleType.LARGEST.toString(), false, null); + } + + BundlesData bundles = superUserAdmin.namespaces().getBundles(namespace); + Assert.assertEquals(bundles.getNumBundles(), 4); + List boundaries = bundles.getBoundaries(); + Assert.assertEquals(boundaries.size(), 5); + + List bundleRanges = new ArrayList<>(); + for (int i = 0; i < boundaries.size() - 1; i++) { + String bundleRange = boundaries.get(i) + "_" + boundaries.get(i + 1); + List allTopicsFromNamespaceBundle = getPulsarService().getBrokerService() + .getAllTopicsFromNamespaceBundle(namespace, namespace + "/" + bundleRange); + System.out.println(StringUtils.join(allTopicsFromNamespaceBundle)); + if (allTopicsFromNamespaceBundle.isEmpty()) { + bundleRanges.add(bundleRange); + } + } + + // test super admin + superUserAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(0)); + + // test tenant manager + tenantManagerAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); + + setAuthorizationOperationChecker(subject, NamespaceOperation.DELETE_BUNDLE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + } + + @Test + public void testPermission() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + final String role = "sub"; + final AuthAction testAction = AuthAction.consume; + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + + // test super admin + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction)); + Map> permissions = superUserAdmin.namespaces().getPermissions(namespace); + Assert.assertEquals(permissions.get(role), Set.of(testAction)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, role); + permissions = superUserAdmin.namespaces().getPermissions(namespace); + Assert.assertTrue(permissions.isEmpty()); + + // test tenant manager + tenantManagerAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction)); + permissions = tenantManagerAdmin.namespaces().getPermissions(namespace); + Assert.assertEquals(permissions.get(role), Set.of(testAction)); + tenantManagerAdmin.namespaces().revokePermissionsOnNamespace(namespace, role); + permissions = tenantManagerAdmin.namespaces().getPermissions(namespace); + Assert.assertTrue(permissions.isEmpty()); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissions(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissions(namespace)); + setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testPermissionOnSubscription() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String subscription = "my-sub"; + final String role = "sub"; + pulsarClient.newConsumer().topic(topic) + .subscriptionName(subscription) + .subscribe().close(); + + + // test super admin + superUserAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role)); + Map> permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertEquals(permissionOnSubscription.get(subscription), Set.of(role)); + superUserAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role); + permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertTrue(permissionOnSubscription.isEmpty()); + + // test tenant manager + tenantManagerAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role)); + permissionOnSubscription = tenantManagerAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertEquals(permissionOnSubscription.get(subscription), Set.of(role)); + tenantManagerAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role); + permissionOnSubscription = tenantManagerAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertTrue(permissionOnSubscription.isEmpty()); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); + setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testClearBacklog() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test super admin + superUserAdmin.namespaces().clearNamespaceBacklog(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().clearNamespaceBacklog(namespace); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBacklog(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().clearNamespaceBacklog(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBacklog(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testClearNamespaceBundleBacklog() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + + // test tenant manager + tenantManagerAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnsubscribeNamespace() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + pulsarClient.newConsumer().topic(topic) + .subscriptionName("sub") + .subscribe().close(); + + // test super admin + superUserAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + + // test tenant manager + tenantManagerAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespace(namespace, "sub")); + + setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespace(namespace, "sub")); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnsubscribeNamespaceBundle() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + pulsarClient.newConsumer().topic(topic) + .subscriptionName("sub") + .subscribe().close(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + + // test tenant manager + tenantManagerAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub")); + + setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub")); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testPackageAPI() throws Exception { + final String namespace = "public/default"; + + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + + File file = File.createTempFile("package-api-test", ".package"); + + // testing upload api + String packageName = "function://public/default/test@v1"; + PackageMetadata originalMetadata = PackageMetadata.builder().description("test").build(); + superUserAdmin.packages().upload(originalMetadata, packageName, file.getPath()); + + // testing download api + String downloadPath = new File(file.getParentFile(), "package-api-test-download.package").getPath(); + superUserAdmin.packages().download(packageName, downloadPath); + File downloadFile = new File(downloadPath); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + List packages = superUserAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + List versions = superUserAdmin.packages().listPackageVersions(packageName); + assertEquals(versions.size(), 1); + assertEquals(versions.get(0), "v1"); + + // testing get packages api + PackageMetadata metadata = superUserAdmin.packages().getMetadata(packageName); + assertEquals(metadata.getDescription(), originalMetadata.getDescription()); + assertNull(metadata.getContact()); + assertTrue(metadata.getModificationTime() > 0); + assertTrue(metadata.getCreateTime() > 0); + assertNull(metadata.getProperties()); + + // testing update package metadata api + PackageMetadata updatedMetadata = originalMetadata; + updatedMetadata.setContact("test@apache.org"); + updatedMetadata.setProperties(Collections.singletonMap("key", "value")); + superUserAdmin.packages().updateMetadata(packageName, updatedMetadata); + + superUserAdmin.packages().getMetadata(packageName); + + // ---- test tenant manager --- + + file = File.createTempFile("package-api-test", ".package"); + + // test tenant manager + packageName = "function://public/default/test@v2"; + originalMetadata = PackageMetadata.builder().description("test").build(); + tenantManagerAdmin.packages().upload(originalMetadata, packageName, file.getPath()); + + // testing download api + downloadPath = new File(file.getParentFile(), "package-api-test-download.package").getPath(); + tenantManagerAdmin.packages().download(packageName, downloadPath); + downloadFile = new File(downloadPath); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + packages = tenantManagerAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + tenantManagerAdmin.packages().listPackageVersions(packageName); + + // testing get packages api + tenantManagerAdmin.packages().getMetadata(packageName); + + // testing update package metadata api + updatedMetadata = originalMetadata; + updatedMetadata.setContact("test@apache.org"); + updatedMetadata.setProperties(Collections.singletonMap("key", "value")); + tenantManagerAdmin.packages().updateMetadata(packageName, updatedMetadata); + + // ---- test nobody --- + + File file3 = File.createTempFile("package-api-test", ".package"); + + // test tenant manager + String packageName3 = "function://public/default/test@v3"; + PackageMetadata originalMetadata3 = PackageMetadata.builder().description("test").build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().upload(originalMetadata3, packageName3, file3.getPath())); + + + // testing download api + String downloadPath3 = new File(file3.getParentFile(), "package-api-test-download.package").getPath(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().download(packageName3, downloadPath3)); + + // testing list packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackages("function", "public/default")); + + // testing list versions api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackageVersions(packageName3)); + + // testing get packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().getMetadata(packageName3)); + + // testing update package metadata api + PackageMetadata updatedMetadata3 = originalMetadata; + updatedMetadata3.setContact("test@apache.org"); + updatedMetadata3.setProperties(Collections.singletonMap("key", "value")); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().updateMetadata(packageName3, updatedMetadata3)); + + + setAuthorizationOperationChecker(subject, NamespaceOperation.PACKAGES); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + File file4 = File.createTempFile("package-api-test", ".package"); + String packageName4 = "function://public/default/test@v4"; + PackageMetadata originalMetadata4 = PackageMetadata.builder().description("test").build(); + String downloadPath4 = new File(file3.getParentFile(), "package-api-test-download.package").getPath(); + if (AuthAction.packages == action) { + subAdmin.packages().upload(originalMetadata4, packageName4, file.getPath()); + + // testing download api + subAdmin.packages().download(packageName4, downloadPath4); + downloadFile = new File(downloadPath4); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + packages = subAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + subAdmin.packages().listPackageVersions(packageName4); + + // testing get packages api + subAdmin.packages().getMetadata(packageName4); + + // testing update package metadata api + PackageMetadata updatedMetadata4 = originalMetadata; + updatedMetadata4.setContact("test@apache.org"); + updatedMetadata4.setProperties(Collections.singletonMap("key", "value")); + subAdmin.packages().updateMetadata(packageName, updatedMetadata4); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().upload(originalMetadata4, packageName4, file4.getPath())); + + // testing download api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().download(packageName4, downloadPath4)); + + // testing list packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackages("function", "public/default")); + + // testing list versions api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackageVersions(packageName4)); + + // testing get packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().getMetadata(packageName4)); + + // testing update package metadata api + PackageMetadata updatedMetadata4 = originalMetadata; + updatedMetadata4.setContact("test@apache.org"); + updatedMetadata4.setProperties(Collections.singletonMap("key", "value")); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().updateMetadata(packageName4, updatedMetadata4)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + } } From 9555504ee2c7adf9febddc585a699a1fdb724013 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 9 Apr 2024 22:43:35 +0800 Subject: [PATCH 06/55] [improve][admin] Add authorization test for schema and align auth for transaction (#22399) --- .../pulsar/broker/admin/TopicAuthZTest.java | 249 ++++++++++++++++++ .../security/MockedPulsarStandalone.java | 4 +- 2 files changed, 252 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 2e75b59ec8582..d09bc0a3ffde1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,19 +20,27 @@ package org.apache.pulsar.broker.admin; import io.jsonwebtoken.Jwts; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.security.MockedPulsarStandalone; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -62,7 +70,9 @@ public class TopicAuthZTest extends MockedPulsarStandalone { public void before() { configureTokenAuthentication(); configureDefaultAuthorization(); + enableTransaction(); start(); + createTransactionCoordinatorAssign(16); this.superUserAdmin =PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) @@ -74,8 +84,18 @@ public void before() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); + + superUserAdmin.tenants().createTenant("pulsar", tenantInfo); + superUserAdmin.namespaces().createNamespace("pulsar/system"); } + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { + getPulsarService().getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } @SneakyThrows @AfterClass(alwaysRun = true) @@ -1086,6 +1106,235 @@ public void testExpireMessageByPosition() { deleteTopic(topic, false); } + public enum OperationAuthType { + Lookup, + Produce, + Consume, + AdminOrSuperUser, + NOAuth + } + + private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(T t) throws PulsarAdminException; + } + + @DataProvider(name = "authFunction") + public Object[][] authFunction () throws Exception { + String sub = "my-sub"; + createTopic(testTopic, false); + @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .enableTransaction(true) + .build(); + @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); + + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(testTopic) + .subscriptionName(sub) + .subscribe(); + + Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) + .build().get(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); + + consumer.acknowledgeAsync(messageId, transaction).get(); + + return new Object[][]{ + // SCHEMA + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( + testTopic, 0), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( + testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, + SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.Produce + }, + // TODO: improve the authorization check for testCompatibility and deleteSchema + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( + testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( + testTopic), + OperationAuthType.AdminOrSuperUser + }, + + // TRANSACTION + + // Modify transaction coordinator + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .abortTransaction(transaction.getTxnID()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .scaleTransactionCoordinators(17), + OperationAuthType.AdminOrSuperUser + }, + // TODO: fix authorization check of check transaction coordinator stats. + // Check transaction coordinator stats + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorInternalStats(1, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorStats(), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionMetadata(transaction.getTxnID()), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .listTransactionCoordinators(), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactions(5, TimeUnit.SECONDS), + OperationAuthType.AdminOrSuperUser + }, + + // TODO: Check the authorization of the topic when get stats of TB or TP + // Check stats related to transaction buffer and transaction pending ack + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckInternalStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), + messageId.getEntryId(), null), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferInternalStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), + OperationAuthType.NOAuth + }, + }; + } + + @Test(dataProvider = "authFunction") + public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) + throws Exception { + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test tenant manager + if (topicOpType != OperationAuthType.AdminOrSuperUser) { + adminConsumer.accept(tenantManagerAdmin); + } + + if (topicOpType != OperationAuthType.NOAuth) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); + + if (authActionMatchOperation(topicOpType, action)) { + adminConsumer.accept(subAdmin); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + superUserAdmin.topics().revokePermissions(testTopic, subject); + } + } + + + private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { + switch (operationAuthType) { + case Lookup -> { + if (AuthAction.consume == action || AuthAction.produce == action) { + return true; + } + } + case Consume -> { + if (AuthAction.consume == action) { + return true; + } + } + case Produce -> { + if (AuthAction.produce == action) { + return true; + } + } + case AdminOrSuperUser -> { + return false; + } + case NOAuth -> { + return true; + } + } + return false; + } + private void createTopic(String topic, boolean partitioned) throws Exception { if (partitioned) { superUserAdmin.topics().createPartitionedTopic(topic, 2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java index 421727c0ed7f4..b82f3b584065d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java @@ -105,7 +105,9 @@ protected void configureTokenAuthentication() { } - + protected void enableTransaction() { + serviceConfiguration.setTransactionCoordinatorEnabled(true); + } protected void configureDefaultAuthorization() { serviceConfiguration.setAuthorizationEnabled(true); From f3d14a6b0b15f6d3c17509b21b28a586a22e5d89 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 9 Apr 2024 07:48:57 -0700 Subject: [PATCH 07/55] [improve][test] Replace usage of curl in Java test and fix stream leaks (#22463) --- .../pulsar/broker/web/WebServiceTest.java | 69 +++++++++---------- 1 file changed, 33 insertions(+), 36 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 72437fe33743e..d2b59ed0e4997 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -23,18 +23,17 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.common.io.CharStreams; import com.google.common.io.Closeables; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; - import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.net.HttpURLConnection; import java.net.URL; import java.security.KeyStore; import java.security.PrivateKey; @@ -361,68 +360,66 @@ public void testBrokerReady() throws Exception { @Test public void testCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", "-H", "Accept-Encoding: gzip", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept-Encoding", "gzip"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); - - InputStream inputStream = process.getInputStream(); - - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); + StringBuilder content = new StringBuilder(); - // Process the decompressed content - StringBuilder content = new StringBuilder(); - int data; - while ((data = gzipInputStream.read()) != -1) { - content.append((char) data); + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + // Process the decompressed content + int data; + while ((data = gzipInputStream.read()) != -1) { + content.append((char) data); + } } - log.info("Response Content: {}", content); - process.waitFor(); + log.info("Response Content: {}", content); assertTrue(content.toString().contains("process_cpu_seconds_total")); } catch (IOException e) { log.error("Failed to decompress the content, likely the content is not compressed ", e); fail(); + } finally { + connection.disconnect(); } } @Test public void testUnCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); + StringBuilder content = new StringBuilder(); - InputStream inputStream = process.getInputStream(); - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); - fail(); - } catch (IOException e) { - log.error("Failed to decompress the content, likely the content is not compressed ", e); - assertTrue(e instanceof ZipException); - } + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + fail(); + } catch (IOException e) { + assertTrue(e instanceof ZipException); + } - BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); - StringBuilder content = new StringBuilder(); - String line; - while ((line = reader.readLine()) != null) { - content.append(line + "\n"); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + String line; + while ((line = reader.readLine()) != null) { + content.append(line + "\n"); + } + } finally { + connection.disconnect(); } log.info("Response Content: {}", content); - process.waitFor(); assertTrue(content.toString().contains("process_cpu_seconds_total")); } From fb5caeb2cd3353db0499e32e9ec79390741b809c Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 10 Apr 2024 04:27:22 +0800 Subject: [PATCH 08/55] [fix][broker] Update topic partition failed when config maxNumPartitionsPerPartitionedTopic<0 (#22397) --- .../pulsar/broker/ServiceConfiguration.java | 3 +- .../admin/impl/PersistentTopicsBase.java | 2 +- .../broker/admin/PersistentTopicsTest.java | 45 +++++++++++++++++++ 3 files changed, 48 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 324a4c9a8dc01..a7deda752fdde 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 @@ -1344,7 +1344,8 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece category = CATEGORY_SERVER, dynamic = true, doc = "The number of partitions per partitioned topic.\n" - + "If try to create or update partitioned topics by exceeded number of partitions, then fail." + + "If try to create or update partitioned topics by exceeded number of partitions, then fail.\n" + + "Use 0 or negative number to disable the check." ) private int maxNumPartitionsPerPartitionedTopic = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index c9c29271b6afe..ab74b1e2bcc0e 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 @@ -348,7 +348,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean } int brokerMaximumPartitionsPerTopic = pulsarService.getConfiguration() .getMaxNumPartitionsPerPartitionedTopic(); - if (brokerMaximumPartitionsPerTopic != 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { + if (brokerMaximumPartitionsPerTopic > 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { throw new RestException(422 /* Unprocessable entity*/, String.format("Desired partitions %s can't be greater than the maximum partitions per" + " topic %s.", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 8e1375303ce4c..c588051a0feff 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -1742,6 +1742,51 @@ public void testUpdatePartitionedTopic() verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); partitionedTopicMetadata = metaCaptor.getValue(); Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + // test for configuration maxNumPartitionsPerPartitionedTopic + conf.setMaxNumPartitionsPerPartitionedTopic(4); + response = mock(AsyncResponse.class); + throwableCaptor = ArgumentCaptor.forClass(Throwable.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(throwableCaptor.capture()); + Assert.assertEquals(throwableCaptor.getValue().getMessage(), + "Desired partitions 5 can't be greater than the maximum partitions per topic 4."); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + conf.setMaxNumPartitionsPerPartitionedTopic(-1); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 5); + + conf.setMaxNumPartitionsPerPartitionedTopic(0); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 6); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 6); } @Test From cea1a9ba9b576bf43f0a45ff8d65369b0f2bbb36 Mon Sep 17 00:00:00 2001 From: zhangqian <503837557@qq.com> Date: Wed, 10 Apr 2024 16:51:26 +0800 Subject: [PATCH 09/55] [fix][broker] Fix message drop record in producer stat (#22458) Co-authored-by: ceceezhang --- .../main/java/org/apache/pulsar/broker/service/Producer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index 7e4459505a523..9cfde67802bb0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -749,7 +749,7 @@ public void updateRates() { } if (this.isNonPersistentTopic) { msgDrop.calculateRate(); - ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getRate(); + ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getValueRate(); } } From b42d94121c0209c197339f1fe6ad702e9880c5f9 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 11 Apr 2024 10:10:31 -0700 Subject: [PATCH 10/55] [improve][broker] Recover susbcription creation on the broken schema ledger topic (#22469) --- .../pulsar/broker/service/ServerCnx.java | 4 +- .../schema/BookkeeperSchemaStorage.java | 2 + .../org/apache/pulsar/schema/SchemaTest.java | 76 +++++++++++++++++++ 3 files changed, 81 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 716f3a1a04c25..4ee6ac43465f4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -26,6 +26,7 @@ import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.unsafeGetPartitionedTopicMetadataAsync; import static org.apache.pulsar.broker.lookup.TopicLookupBase.lookupTopicAsync; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.getMigratedClusterUrl; +import static org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.ignoreUnrecoverableBKException; import static org.apache.pulsar.common.api.proto.ProtocolVersion.v5; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.apache.pulsar.common.protocol.Commands.newCloseConsumer; @@ -1291,7 +1292,8 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { .schemaType(schema == null ? null : schema.getType()) .build(); if (schema != null && schema.getType() != SchemaType.AUTO_CONSUME) { - return topic.addSchemaIfIdleOrCheckCompatible(schema) + return ignoreUnrecoverableBKException + (topic.addSchemaIfIdleOrCheckCompatible(schema)) .thenCompose(v -> topic.subscribe(option)); } else { return topic.subscribe(option); 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 c509764bf6710..acdd906f6b8af 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -52,6 +52,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.protocol.schema.SchemaVersion; @@ -716,6 +717,7 @@ public static CompletableFuture ignoreUnrecoverableBKException(Completabl return source.exceptionally(t -> { if (t.getCause() != null && (t.getCause() instanceof SchemaException) + && !(t.getCause() instanceof IncompatibleSchemaException) && !((SchemaException) t.getCause()).isRecoverable()) { // Meeting NoSuchLedgerExistsException, NoSuchEntryException or // NoSuchLedgerExistsOnMetadataServerException when reading schemas in diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index aa47c378fc38c..d21e853ba0982 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -32,6 +32,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.Sets; import java.io.ByteArrayInputStream; +import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -58,6 +59,8 @@ import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; import org.apache.pulsar.broker.service.schema.SchemaRegistry; import org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.SchemaLocator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -87,6 +90,9 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataSerde; +import org.apache.pulsar.metadata.api.Stat; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -1410,4 +1416,74 @@ public User(String name) { } } + /** + * This test validates that consumer/producers should recover on topic whose + * schema ledgers are not able to open due to non-recoverable error. + * + * @throws Exception + */ + @Test + public void testDeletedSchemaLedgerRecovery() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "test-multi-version-schema-one"; + final String subName = "test"; + final String topicName = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet(CLUSTER_NAME)); + + // (1) create schema + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + + Schemas.PersonTwo personTwo = new Schemas.PersonTwo(); + personTwo.setId(1); + personTwo.setName("Tom"); + + Consumer consumer = pulsarClient + .newConsumer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .subscriptionName(subName).topic(topicName).subscribe(); + + producer.send(personTwo); + producer.close(); + consumer.close(); + + // (2) Delete schema ledger + MetadataCache locatorEntryCache = pulsar.getLocalMetadataStore() + .getMetadataCache(new MetadataSerde() { + @Override + public byte[] serialize(String path, SchemaStorageFormat.SchemaLocator value) { + return value.toByteArray(); + } + + @Override + public SchemaStorageFormat.SchemaLocator deserialize(String path, byte[] content, Stat stat) + throws IOException { + return SchemaStorageFormat.SchemaLocator.parseFrom(content); + } + }); + String path = "/schemas/public/" + namespace + "/test-multi-version-schema-one"; + SchemaLocator schema = locatorEntryCache.get(path).get().get(); + schema = locatorEntryCache.get(path).get().get(); + long ledgerId = schema.getInfo().getPosition().getLedgerId(); + pulsar.getBookKeeperClient().deleteLedger(ledgerId); + + // (3) Topic should recover from deleted schema and should allow to create consumer and producer + consumer = pulsarClient + .newConsumer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .subscriptionName(subName).topic(topicName).subscribe(); + + producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + assertNotNull(consumer); + assertNotNull(producer); + consumer.close(); + producer.close(); + } } From 094742d5fa6f07b5ceed581876c45564fa0379bd Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 11 Apr 2024 16:21:45 -0700 Subject: [PATCH 11/55] [fix][broker] Do not migrate internal topics during Blue-Green Migration when ExtensibleLoadBalancer is used (#22478) --- .../pulsar/PulsarClusterMetadataSetup.java | 4 +- .../extensions/ExtensibleLoadManagerImpl.java | 4 +- .../channel/ServiceUnitStateChannelImpl.java | 4 ++ .../pulsar/broker/service/BrokerService.java | 3 ++ .../nonpersistent/NonPersistentTopic.java | 4 ++ .../service/persistent/PersistentTopic.java | 4 ++ .../broker/service/ClusterMigrationTest.java | 47 +++++++++++++++---- 7 files changed, 57 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index e8efeabcdd37c..d5b8df43a4737 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -358,8 +358,8 @@ private static void initializeCluster(Arguments arguments, int bundleNumberForDe log.info("Cluster metadata for '{}' setup correctly", arguments.cluster); } - static void createTenantIfAbsent(PulsarResources resources, String tenant, String cluster) throws IOException, - InterruptedException, ExecutionException { + public static void createTenantIfAbsent(PulsarResources resources, String tenant, String cluster) + throws IOException, InterruptedException, ExecutionException { TenantResources tenantResources = resources.getTenantResources(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index c35dc11d7efc7..0c9448ab69c38 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -825,11 +825,11 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.await(); if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; } - initWaiter.await(); // Confirm the system topics have been created or create them if they do not exist. // If the leader has changed, the new leader need to reset // the local brokerService.topics (by this topic creations). @@ -875,11 +875,11 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.await(); if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; } - initWaiter.await(); unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); brokerLoadDataStore.init(); 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 71ddb3acb28b7..68b38080e73a1 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 @@ -293,6 +293,10 @@ public synchronized void start() throws PulsarServerException { log.info("Closed the channel producer."); } } + + PulsarClusterMetadataSetup.createTenantIfAbsent + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), config.getClusterName()); + PulsarClusterMetadataSetup.createNamespaceIfAbsent (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 549dfef896cd0..b4d0f38b4a4dc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1785,6 +1785,9 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } private CompletableFuture checkTopicAlreadyMigrated(TopicName topicName) { + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + return CompletableFuture.completedFuture(null); + } CompletableFuture result = new CompletableFuture<>(); AbstractTopic.isClusterMigrationEnabled(pulsar, topicName.toString()).handle((isMigrated, ex) -> { if (isMigrated) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 88f8c69895002..0ac06d6883ff1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -996,6 +996,10 @@ public boolean isActive() { @Override public CompletableFuture checkClusterMigration() { + if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + return CompletableFuture.completedFuture(null); + } + Optional url = getMigratedClusterUrl(); if (url.isPresent()) { this.migrated = true; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index b21cd165402e4..3ceecd7f4aa20 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2740,6 +2740,10 @@ private boolean hasBacklogs() { @Override public CompletableFuture checkClusterMigration() { + if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + return CompletableFuture.completedFuture(null); + } + Optional clusterUrl = getMigratedClusterUrl(); if (!clusterUrl.isPresent()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index 7bd82cdd840ee..20e13023cacfb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; @@ -53,6 +55,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "cluster-migration") @@ -86,6 +89,8 @@ public class ClusterMigrationTest { PulsarService pulsar4; PulsarAdmin admin4; + String loadManagerClassName; + @DataProvider(name="NamespaceMigrationTopicSubscriptionTypes") public Object[][] namespaceMigrationSubscriptionTypes() { return new Object[][] { @@ -95,15 +100,28 @@ public Object[][] namespaceMigrationSubscriptionTypes() { }; } + @DataProvider(name = "loadManagerClassName") + public static Object[][] loadManagerClassName() { + return new Object[][]{ + {ModularLoadManagerImpl.class.getName()}, + {ExtensibleLoadManagerImpl.class.getName()} + }; + } + + @Factory(dataProvider = "loadManagerClassName") + public ClusterMigrationTest(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; + } + @BeforeMethod(alwaysRun = true, timeOut = 300000) public void setup() throws Exception { log.info("--- Starting ReplicatorTestBase::setup ---"); - broker1 = new TestBroker("r1"); - broker2 = new TestBroker("r2"); - broker3 = new TestBroker("r3"); - broker4 = new TestBroker("r4"); + broker1 = new TestBroker("r1", loadManagerClassName); + broker2 = new TestBroker("r2", loadManagerClassName); + broker3 = new TestBroker("r3", loadManagerClassName); + broker4 = new TestBroker("r4", loadManagerClassName); pulsar1 = broker1.getPulsarService(); url1 = new URL(pulsar1.getWebServiceAddress()); @@ -163,9 +181,9 @@ public void setup() throws Exception { .brokerServiceUrlTls(pulsar4.getBrokerServiceUrlTls()).build()); // Setting r3 as replication cluster for r1 - admin1.tenants().createTenant("pulsar", + updateTenantInfo(admin1, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r3"))); - admin3.tenants().createTenant("pulsar", + updateTenantInfo(admin3, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r3"))); admin1.namespaces().createNamespace(namespace, Sets.newHashSet("r1", "r3")); admin3.namespaces().createNamespace(namespace); @@ -175,9 +193,9 @@ public void setup() throws Exception { admin1.namespaces().setNamespaceReplicationClusters(namespaceNotToMigrate, Sets.newHashSet("r1", "r3")); // Setting r4 as replication cluster for r2 - admin2.tenants().createTenant("pulsar", + updateTenantInfo(admin2, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r2", "r4"))); - admin4.tenants().createTenant("pulsar", + updateTenantInfo(admin4,"pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r2", "r4"))); admin2.namespaces().createNamespace(namespace, Sets.newHashSet("r2", "r4")); admin4.namespaces().createNamespace(namespace); @@ -200,6 +218,14 @@ public void setup() throws Exception { } + protected void updateTenantInfo(PulsarAdmin admin, String tenant, TenantInfoImpl tenantInfo) throws Exception { + if (!admin.tenants().getTenants().contains(tenant)) { + admin.tenants().createTenant(tenant, tenantInfo); + } else { + admin.tenants().updateTenant(tenant, tenantInfo); + } + } + @AfterMethod(alwaysRun = true, timeOut = 300000) protected void cleanup() throws Exception { log.info("--- Shutting down ---"); @@ -1059,9 +1085,11 @@ public void testNamespaceMigrationWithReplicationBacklog(SubscriptionType subTyp static class TestBroker extends MockedPulsarServiceBaseTest { private String clusterName; + private String loadManagerClassName; - public TestBroker(String clusterName) throws Exception { + public TestBroker(String clusterName, String loadManagerClassName) throws Exception { this.clusterName = clusterName; + this.loadManagerClassName = loadManagerClassName; setup(); } @@ -1073,6 +1101,7 @@ protected void setup() throws Exception { @Override protected void doInitConf() throws Exception { super.doInitConf(); + this.conf.setLoadManagerClassName(loadManagerClassName); this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setBrokerServicePortTls(Optional.of(0)); } From 4a5400f0c66dab2c3fbb0050c8f537952fef1951 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 00:15:33 -0700 Subject: [PATCH 12/55] [improve][misc] Upgrade to Bookkeeper 4.16.5 (#22484) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 518f92313753f..a409ad07ed1b4 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -356,34 +356,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.4.jar - - org.apache.bookkeeper-circe-checksum-4.16.4.jar - - org.apache.bookkeeper-cpu-affinity-4.16.4.jar - - org.apache.bookkeeper-statelib-4.16.4.jar - - org.apache.bookkeeper-stream-storage-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-common-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.4.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.4.jar - - org.apache.bookkeeper-stream-storage-server-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.4.jar - - org.apache.bookkeeper.http-http-server-4.16.4.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.4.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.4.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.4.jar - - org.apache.distributedlog-distributedlog-common-4.16.4.jar - - org.apache.distributedlog-distributedlog-core-4.16.4-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.4.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.4.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.4.jar - - org.apache.bookkeeper-native-io-4.16.4.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar + - org.apache.bookkeeper-circe-checksum-4.16.5.jar + - org.apache.bookkeeper-cpu-affinity-4.16.5.jar + - org.apache.bookkeeper-statelib-4.16.5.jar + - org.apache.bookkeeper-stream-storage-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-common-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar + - org.apache.bookkeeper-stream-storage-server-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar + - org.apache.bookkeeper.http-http-server-4.16.5.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar + - org.apache.distributedlog-distributedlog-common-4.16.5.jar + - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.5.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar + - org.apache.bookkeeper-native-io-4.16.5.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index b5036b67751f0..3ac489fa49a68 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -393,9 +393,9 @@ The Apache Software License, Version 2.0 - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - - bookkeeper-common-allocator-4.16.4.jar - - cpu-affinity-4.16.4.jar - - circe-checksum-4.16.4.jar + - bookkeeper-common-allocator-4.16.5.jar + - cpu-affinity-4.16.5.jar + - circe-checksum-4.16.5.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index 835bd28f7f25b..47ac21b62bfed 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.4 + 4.16.5 3.9.2 1.5.0 1.10.0 From 7984cc2f93f8dc85b598ded1167508eae4ee06ec Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 05:56:55 -0700 Subject: [PATCH 13/55] [feat][admin] Enable Gzip Compression by Default in Admin Client (#22464) --- .../client/admin/PulsarAdminBuilder.java | 11 +- pulsar-client-admin/pom.xml | 7 + .../internal/PulsarAdminBuilderImpl.java | 22 +++- .../admin/internal/PulsarAdminImpl.java | 8 +- .../internal/http/AsyncHttpConnector.java | 14 +- .../http/AsyncHttpConnectorProvider.java | 9 +- .../admin/internal/PulsarAdminGzipTest.java | 122 ++++++++++++++++++ 7 files changed, 183 insertions(+), 10 deletions(-) create mode 100644 pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java index 1260555a7c43f..1b025a752d9f3 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java @@ -327,4 +327,13 @@ PulsarAdminBuilder authentication(String authPluginClassName, Maphamcrest test + + + com.github.tomakehurst + wiremock-jre8 + ${wiremock.version} + test + diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index 009fa67fbaa29..f7b1695f5f37b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -38,10 +38,11 @@ public class PulsarAdminBuilderImpl implements PulsarAdminBuilder { protected ClientConfigurationData conf; private ClassLoader clientBuilderClassLoader = null; + private boolean acceptGzipCompression = true; @Override public PulsarAdmin build() throws PulsarClientException { - return new PulsarAdminImpl(conf.getServiceUrl(), conf, clientBuilderClassLoader); + return new PulsarAdminImpl(conf.getServiceUrl(), conf, clientBuilderClassLoader, acceptGzipCompression); } public PulsarAdminBuilderImpl() { @@ -54,13 +55,24 @@ private PulsarAdminBuilderImpl(ClientConfigurationData conf) { @Override public PulsarAdminBuilder clone() { - return new PulsarAdminBuilderImpl(conf.clone()); + PulsarAdminBuilderImpl pulsarAdminBuilder = new PulsarAdminBuilderImpl(conf.clone()); + pulsarAdminBuilder.clientBuilderClassLoader = clientBuilderClassLoader; + pulsarAdminBuilder.acceptGzipCompression = acceptGzipCompression; + return pulsarAdminBuilder; } @Override public PulsarAdminBuilder loadConf(Map config) { conf = ConfigurationDataUtils.loadData(config, conf, ClientConfigurationData.class); setAuthenticationFromPropsIfAvailable(conf); + if (config.containsKey("acceptGzipCompression")) { + Object acceptGzipCompressionObj = config.get("acceptGzipCompression"); + if (acceptGzipCompressionObj instanceof Boolean) { + acceptGzipCompression = (Boolean) acceptGzipCompressionObj; + } else { + acceptGzipCompression = Boolean.parseBoolean(acceptGzipCompressionObj.toString()); + } + } return this; } @@ -227,4 +239,10 @@ public PulsarAdminBuilder setContextClassLoader(ClassLoader clientBuilderClassLo this.clientBuilderClassLoader = clientBuilderClassLoader; return this; } + + @Override + public PulsarAdminBuilder acceptGzipCompression(boolean acceptGzipCompression) { + this.acceptGzipCompression = acceptGzipCompression; + return this; + } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index 259ca90cc08b7..39347850cf69c 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -106,6 +106,12 @@ public class PulsarAdminImpl implements PulsarAdmin { public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigData, ClassLoader clientBuilderClassLoader) throws PulsarClientException { + this(serviceUrl, clientConfigData, clientBuilderClassLoader, true); + } + + public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigData, + ClassLoader clientBuilderClassLoader, boolean acceptGzipCompression) + throws PulsarClientException { checkArgument(StringUtils.isNotBlank(serviceUrl), "Service URL needs to be specified"); this.clientConfigData = clientConfigData; @@ -119,7 +125,7 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa } AsyncHttpConnectorProvider asyncConnectorProvider = new AsyncHttpConnectorProvider(clientConfigData, - clientConfigData.getAutoCertRefreshSeconds()); + clientConfigData.getAutoCertRefreshSeconds(), acceptGzipCompression); ClientConfig httpConfig = new ClientConfig(); httpConfig.property(ClientProperties.FOLLOW_REDIRECTS, true); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index 9ed2b8564f2ae..9ad0ce5029c47 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -83,19 +83,23 @@ public class AsyncHttpConnector implements Connector { private final PulsarServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("delayer")); + private final boolean acceptGzipCompression; - public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds) { + public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + boolean acceptGzipCompression) { this((int) client.getConfiguration().getProperty(ClientProperties.CONNECT_TIMEOUT), (int) client.getConfiguration().getProperty(ClientProperties.READ_TIMEOUT), PulsarAdminImpl.DEFAULT_REQUEST_TIMEOUT_SECONDS * 1000, autoCertRefreshTimeSeconds, - conf); + conf, acceptGzipCompression); } @SneakyThrows public AsyncHttpConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, - int autoCertRefreshTimeSeconds, ClientConfigurationData conf) { + int autoCertRefreshTimeSeconds, ClientConfigurationData conf, + boolean acceptGzipCompression) { + this.acceptGzipCompression = acceptGzipCompression; DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); @@ -339,6 +343,10 @@ private CompletableFuture oneShot(InetSocketAddress host, ClientReques } }); + if (acceptGzipCompression) { + builder.setHeader(HttpHeaders.ACCEPT_ENCODING, "gzip"); + } + return builder.execute().toCompletableFuture(); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java index 4467f77d1f993..d20dc84849458 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java @@ -32,16 +32,19 @@ public class AsyncHttpConnectorProvider implements ConnectorProvider { private final ClientConfigurationData conf; private Connector connector; private final int autoCertRefreshTimeSeconds; + private final boolean acceptGzipCompression; - public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefreshTimeSeconds) { + public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + boolean acceptGzipCompression) { this.conf = conf; this.autoCertRefreshTimeSeconds = autoCertRefreshTimeSeconds; + this.acceptGzipCompression = acceptGzipCompression; } @Override public Connector getConnector(Client client, Configuration runtimeConfig) { if (connector == null) { - connector = new AsyncHttpConnector(client, conf, autoCertRefreshTimeSeconds); + connector = new AsyncHttpConnector(client, conf, autoCertRefreshTimeSeconds, acceptGzipCompression); } return connector; } @@ -50,6 +53,6 @@ public Connector getConnector(Client client, Configuration runtimeConfig) { public AsyncHttpConnector getConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, int autoCertRefreshTimeSeconds) { return new AsyncHttpConnector(connectTimeoutMs, readTimeoutMs, requestTimeoutMs, autoCertRefreshTimeSeconds, - conf); + conf, acceptGzipCompression); } } diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java new file mode 100644 index 0000000000000..2bfa382be1096 --- /dev/null +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.admin.internal; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.absent; +import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; +import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; +import static org.testng.Assert.assertEquals; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.zip.GZIPOutputStream; +import lombok.Cleanup; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class PulsarAdminGzipTest { + WireMockServer server; + + @BeforeClass(alwaysRun = true) + void beforeClass() throws IOException { + server = new WireMockServer(WireMockConfiguration.wireMockConfig() + .port(0)); + server.start(); + } + + @AfterClass(alwaysRun = true) + void afterClass() { + if (server != null) { + server.stop(); + } + } + + static byte[] gzipContent(String content) throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + try(GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) { + gzipOutputStream.write(content.getBytes(StandardCharsets.UTF_8)); + } + return byteArrayOutputStream.toByteArray(); + } + + @AfterMethod + void resetAllMocks() { + server.resetAll(); + } + + @Test + public void testGzipRequestedGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", equalTo("gzip")) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withHeader("Content-Encoding", "gzip") + .withBody(gzipContent("[\"gzip-test\", \"gzip-test2\"]")))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(true) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("gzip-test", "gzip-test2")); + } + + @Test + public void testGzipRequestedNoGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", equalTo("gzip")) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withBody("[\"test\", \"test2\"]"))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(true) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("test", "test2")); + } + + @Test + public void testNoGzipRequestedNoGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", absent()) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withBody("[\"test\", \"test2\"]"))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(false) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("test", "test2")); + } +} From dbe1a4816c12535da2013ed5da5ee7796d8b4638 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 12 Apr 2024 21:52:34 +0800 Subject: [PATCH 14/55] [improve][broker] Reduce the duplicated null check for LeaderElectionService (#22465) --- .../broker/namespace/NamespaceService.java | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 6228703f03ab9..4492f9c809435 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -539,23 +539,12 @@ private CompletableFuture> findBrokerServiceUrl( private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture> lookupFuture, LookupOptions options) { - if (null == pulsar.getLeaderElectionService()) { - LOG.warn("The leader election has not yet been completed! NamespaceBundle[{}]", bundle); - lookupFuture.completeExceptionally( - new IllegalStateException("The leader election has not yet been completed!")); - return; - } String candidateBroker; - LeaderElectionService les = pulsar.getLeaderElectionService(); if (les == null) { - // The leader election service was not initialized yet. This can happen because the broker service is - // initialized first, and it might start receiving lookup requests before the leader election service is - // fully initialized. - LOG.warn("Leader election service isn't initialized yet. " - + "Returning empty result to lookup. NamespaceBundle[{}]", - bundle); - lookupFuture.complete(Optional.empty()); + LOG.warn("The leader election has not yet been completed! NamespaceBundle[{}]", bundle); + lookupFuture.completeExceptionally( + new IllegalStateException("The leader election has not yet been completed!")); return; } From b85730069ee4c5f96406a075e354d0592fdab434 Mon Sep 17 00:00:00 2001 From: Mukesh Kumar <65598381+mukesh154@users.noreply.github.com> Date: Fri, 12 Apr 2024 22:07:28 +0530 Subject: [PATCH 15/55] [improve][broker] backlog quota exceed limit log replaced with `debug` (#22488) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3ceecd7f4aa20..3c9ab04d79a0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3224,14 +3224,14 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba if ((retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold || retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception)) { if (backlogQuotaType == BacklogQuotaType.destination_storage && isSizeBacklogExceeded()) { - log.info("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { return checkTimeBacklogExceeded().thenCompose(isExceeded -> { if (isExceeded) { - log.info("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } else { From d1748573f1cb294838b69b5d80af672c3ee9e453 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:09:54 -0700 Subject: [PATCH 16/55] [fix][test] Fix NPE in BookKeeperClusterTestCase tearDown (#22493) --- .../apache/bookkeeper/test/BookKeeperClusterTestCase.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 0ddd04ebc4830..a323ecfeb8ea6 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -86,7 +86,7 @@ public abstract class BookKeeperClusterTestCase { protected String testName; - @BeforeMethod + @BeforeMethod(alwaysRun = true) public void handleTestMethodName(Method method) { testName = method.getName(); } @@ -148,7 +148,7 @@ public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeo } } - @BeforeTest + @BeforeTest(alwaysRun = true) public void setUp() throws Exception { setUp(getLedgersRootPath()); } @@ -222,7 +222,9 @@ public void tearDown() throws Exception { tearDownException = e; } - executor.shutdownNow(); + if (executor != null) { + executor.shutdownNow(); + } LOG.info("Tearing down test {} in {} ms.", testName, sw.elapsed(TimeUnit.MILLISECONDS)); if (tearDownException != null) { From 15ed6595af5489a007db82002ed3391589bad54d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:35:09 -0700 Subject: [PATCH 17/55] [improve][broker] Improve Gzip compression, allow excluding specific paths or disabling it (#22370) --- .../pulsar/broker/ServiceConfiguration.java | 13 +++++ .../pulsar/broker/web/GzipHandlerUtil.java | 48 +++++++++++++++++++ .../apache/pulsar/broker/web/WebService.java | 10 ++-- .../proxy/server/AdminProxyHandler.java | 1 + 4 files changed, 66 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index a7deda752fdde..38a4c552f0b6b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -331,6 +331,19 @@ public class ServiceConfiguration implements PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_SERVER, doc = + "Gzip compression is enabled by default. Specific paths can be excluded from compression.\n" + + "There are 2 syntaxes supported, Servlet url-pattern based, and Regex based.\n" + + "If the spec starts with '^' the spec is assumed to be a regex based path spec and will match " + + "with normal Java regex rules.\n" + + "If the spec starts with '/' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for either an exact match or prefix based match.\n" + + "If the spec starts with '*.' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for a suffix based match.\n" + + "All other syntaxes are unsupported.\n" + + "Disable all compression with ^.* or ^.*$") + private List httpServerGzipCompressionExcludedPaths = new ArrayList<>(); + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") private boolean delayedDeliveryEnabled = true; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java new file mode 100644 index 0000000000000..37c9c05e5d53c --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.web; + +import java.util.List; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; + +public class GzipHandlerUtil { + public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { + Handler wrappedHandler; + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + // no need to add GZIP handler if it's disabled by setting the excluded path to "^.*" or "^.*$" + wrappedHandler = innerHandler; + } else { + // add GZIP handler which is active when the request contains "Accept-Encoding: gzip" header + GzipHandler gzipHandler = new GzipHandler(); + gzipHandler.setHandler(innerHandler); + if (gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() > 0) { + gzipHandler.setExcludedPaths(gzipCompressionExcludedPaths.toArray(new String[0])); + } + wrappedHandler = gzipHandler; + } + return wrappedHandler; + } + + public static boolean isGzipCompressionCompletelyDisabled(List gzipCompressionExcludedPaths) { + return gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() == 1 + && (gzipCompressionExcludedPaths.get(0).equals("^.*") + || gzipCompressionExcludedPaths.get(0).equals("^.*$")); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index a7c4244899069..8dc36e2917ed1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -44,7 +44,6 @@ import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -268,9 +267,7 @@ public void addServlet(String path, ServletHolder servletHolder, boolean require } filterInitializer.addFilters(servletContextHandler, requiresAuthentication); - GzipHandler gzipHandler = new GzipHandler(); - gzipHandler.setHandler(servletContextHandler); - handlers.add(gzipHandler); + handlers.add(servletContextHandler); } public void addStaticResources(String basePath, String resourcePath) { @@ -294,8 +291,10 @@ public void start() throws PulsarServerException { ContextHandlerCollection contexts = new ContextHandlerCollection(); contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + Handler handlerForContexts = GzipHandlerUtil.wrapWithGzipHandler(contexts, + pulsar.getConfig().getHttpServerGzipCompressionExcludedPaths()); HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); + handlerCollection.setHandlers(new Handler[] {handlerForContexts, new DefaultHandler(), requestLogHandler}); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -306,7 +305,6 @@ public void start() throws PulsarServerException { } catch (IllegalArgumentException e) { // Already registered. Eg: in unit tests } - handlers.add(stats); server.setHandler(stats); server.start(); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index c528ceb2cf5b7..caaa99c5d40cc 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -158,6 +158,7 @@ protected HttpClient createHttpClient() throws ServletException { client.start(); // Content must not be decoded, otherwise the client gets confused. + // Allow encoded content, such as "Content-Encoding: gzip", to pass through without decoding it. client.getContentDecoderFactories().clear(); // Pass traffic to the client, only intercept what's necessary. From 97153dcf6f0f958a2ee2816ea64115a9a9ef4635 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 11:38:42 -0700 Subject: [PATCH 18/55] [fix][test] Move ExtensibleLoadManagerImplTest to flaky tests (#22495) --- .../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 aee57f9d26093..e87d6c994cd76 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 @@ -142,7 +142,7 @@ * Unit test for {@link ExtensibleLoadManagerImpl}. */ @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") @SuppressWarnings("unchecked") public class ExtensibleLoadManagerImplTest extends ExtensibleLoadManagerImplBaseTest { From 51ecd0235ce5d5ad03c563a3338b29c6a117d216 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 13 Apr 2024 08:58:53 +0800 Subject: [PATCH 19/55] [cleanup][broker] Remove unused NamespaceBundleFactory parameter when creating OwnershipCache (#22482) --- .../broker/namespace/NamespaceService.java | 2 +- .../broker/namespace/OwnershipCache.java | 4 +--- .../broker/namespace/OwnershipCacheTest.java | 19 ++++++++----------- 3 files changed, 10 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 4492f9c809435..7c62f264c78d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -192,7 +192,7 @@ public NamespaceService(PulsarService pulsar) { this.config = pulsar.getConfiguration(); this.loadManager = pulsar.getLoadManager(); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); - this.ownershipCache = new OwnershipCache(pulsar, bundleFactory, this); + this.ownershipCache = new OwnershipCache(pulsar, this); this.namespaceClients = ConcurrentOpenHashMap.newBuilder().build(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 0033abf36c78c..9a4534f538774 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -36,7 +36,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.naming.NamespaceBundle; -import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.coordination.LockManager; @@ -115,8 +114,7 @@ public CompletableFuture asyncLoad(NamespaceBundle namespaceBundle, * * the local broker URL that will be set as owner for the ServiceUnit */ - public OwnershipCache(PulsarService pulsar, NamespaceBundleFactory bundleFactory, - NamespaceService namespaceService) { + public OwnershipCache(PulsarService pulsar, NamespaceService namespaceService) { this.namespaceService = namespaceService; this.pulsar = pulsar; this.ownerBrokerUrl = pulsar.getBrokerServiceUrl(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java index c92127457aaf2..2c3182659f022 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java @@ -55,15 +55,12 @@ import org.apache.pulsar.metadata.coordination.impl.CoordinationServiceImpl; import org.apache.pulsar.zookeeper.ZookeeperServerTest; import org.awaitility.Awaitility; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Test(groups = "broker") public class OwnershipCacheTest { - private static final Logger log = LoggerFactory.getLogger(OwnershipCacheTest.class); private PulsarService pulsar; private ServiceConfiguration config; @@ -123,14 +120,14 @@ public void teardown() throws Exception { @Test public void testConstructor() { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); assertNotNull(cache); assertNotNull(cache.getOwnedBundles()); } @Test public void testDisableOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-1"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -148,7 +145,7 @@ public void testDisableOwnership() throws Exception { @Test public void testGetOrSetOwner() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testFullBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-2"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); @@ -194,7 +191,7 @@ public void testGetOrSetOwner() throws Exception { @Test public void testGetOwner() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-3"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); @@ -241,7 +238,7 @@ public void testGetOwner() throws Exception { @Test public void testGetOwnedServiceUnit() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-5"); NamespaceBundle testBundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -301,7 +298,7 @@ public void testGetOwnedServiceUnit() throws Exception { @Test public void testGetOwnedServiceUnits() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-6"); NamespaceBundle testBundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -347,7 +344,7 @@ public void testGetOwnedServiceUnits() throws Exception { @Test public void testRemoveOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-7"); NamespaceBundle bundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -373,7 +370,7 @@ public void testRemoveOwnership() throws Exception { @Test public void testReestablishOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testFullBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-8"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); From 7009071b6d53bbc3d740ea99cdc0c010692679ab Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 13 Apr 2024 10:00:23 -0700 Subject: [PATCH 20/55] [fix][broker] Optimize /metrics, fix unbounded request queue issue and fix race conditions in metricsBufferResponse mode (#22494) --- conf/proxy.conf | 6 +- .../PrometheusMetricsGeneratorUtils.java | 2 +- .../prometheus/PrometheusMetricsServlet.java | 149 +++++--- .../pulsar/broker/stats/TimeWindow.java | 94 ----- .../pulsar/broker/stats/WindowWrap.java | 56 --- .../stats/prometheus/MetricsExports.java | 68 ++++ .../prometheus/PrometheusMetricStreams.java | 2 +- .../PrometheusMetricsGenerator.java | 328 +++++++++++------- .../PulsarPrometheusMetricsServlet.java | 140 +++++++- .../broker/stats/prometheus/TopicStats.java | 12 +- .../pulsar/PrometheusMetricsTestUtil.java | 84 +++++ .../persistent/BucketDelayedDeliveryTest.java | 6 +- .../persistent/PersistentTopicTest.java | 4 +- .../service/schema/SchemaServiceTest.java | 4 +- .../broker/stats/ConsumerStatsTest.java | 4 +- .../broker/stats/MetadataStoreStatsTest.java | 6 +- .../broker/stats/PrometheusMetricsTest.java | 120 ++++--- .../broker/stats/SubscriptionStatsTest.java | 4 +- .../pulsar/broker/stats/TimeWindowTest.java | 83 ----- .../broker/stats/TransactionMetricsTest.java | 18 +- .../buffer/TransactionBufferClientTest.java | 4 +- .../pendingack/PendingAckPersistentTest.java | 4 +- .../pulsar/broker/web/WebServiceTest.java | 4 +- .../common/util/SimpleTextOutputStream.java | 16 +- .../proxy/server/ProxyConfiguration.java | 6 + .../pulsar/proxy/server/ProxyService.java | 3 +- .../proxy/server/ProxyServiceStarter.java | 40 ++- 27 files changed, 739 insertions(+), 528 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java diff --git a/conf/proxy.conf b/conf/proxy.conf index 8285e1cb75320..5a9d433f39ceb 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -376,5 +376,7 @@ zooKeeperCacheExpirySeconds=-1 enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true -# Enable cache metrics data, default value is false -metricsBufferResponse=false +# Time in milliseconds that metrics endpoint would time out. Default is 30s. +# Set it to 0 to disable timeout. +metricsServletTimeoutMs=30000 + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java index 828d9871bb3de..077d5280b5102 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java @@ -76,7 +76,7 @@ public static void generateSystemMetrics(SimpleTextOutputStream stream, String c } for (int j = 0; j < sample.labelNames.size(); j++) { String labelValue = sample.labelValues.get(j); - if (labelValue != null) { + if (labelValue != null && labelValue.indexOf('"') > -1) { labelValue = labelValue.replace("\"", "\\\""); } if (j > 0) { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 64d1fcdab6f14..8a41bed29d44f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -25,9 +25,13 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,67 +39,132 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { - private static final long serialVersionUID = 1L; - private static final int HTTP_STATUS_OK_200 = 200; - private static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; - - private final long metricsServletTimeoutMs; - private final String cluster; + static final int HTTP_STATUS_OK_200 = 200; + static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; + protected final long metricsServletTimeoutMs; + protected final String cluster; protected List metricsProviders; - private ExecutorService executor = null; + protected ExecutorService executor = null; + protected final int executorMaxThreads; public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster) { + this(metricsServletTimeoutMs, cluster, 1); + } + + public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster, int executorMaxThreads) { this.metricsServletTimeoutMs = metricsServletTimeoutMs; this.cluster = cluster; + this.executorMaxThreads = executorMaxThreads; } @Override public void init() throws ServletException { - executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("prometheus-stats")); + if (executorMaxThreads > 0) { + executor = + Executors.newScheduledThreadPool(executorMaxThreads, new DefaultThreadFactory("prometheus-stats")); + } } @Override protected void doGet(HttpServletRequest request, HttpServletResponse response) { AsyncContext context = request.startAsync(); - context.setTimeout(metricsServletTimeoutMs); - executor.execute(() -> { - long start = System.currentTimeMillis(); - HttpServletResponse res = (HttpServletResponse) context.getResponse(); - try { - res.setStatus(HTTP_STATUS_OK_200); - res.setContentType("text/plain;charset=utf-8"); - generateMetrics(cluster, res.getOutputStream()); - } catch (Exception e) { - long end = System.currentTimeMillis(); - long time = end - start; - if (e instanceof EOFException) { - // NO STACKTRACE - log.error("Failed to send metrics, " - + "likely the client or this server closed " - + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); - } else { - log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean taskStarted = new AtomicBoolean(false); + Future future = executor.submit(() -> { + taskStarted.set(true); + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos(metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); } - res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); - } finally { - long end = System.currentTimeMillis(); - long time = end - start; - try { - context.complete(); - } catch (IllegalStateException e) { - // this happens when metricsServletTimeoutMs expires - // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled - log.error("Failed to generate prometheus stats, " - + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + context.complete(); + return; + } + handleAsyncMetricsRequest(context); + }); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); } } + + @Override + public void onTimeout(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + log.warn("Prometheus metrics request timed out"); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + } + + @Override + public void onStartAsync(AsyncEvent asyncEvent) throws IOException { + + } }); + + } + + private void handleAsyncMetricsRequest(AsyncContext context) { + long start = System.currentTimeMillis(); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + try { + generateMetricsSynchronously(res); + } catch (Exception e) { + long end = System.currentTimeMillis(); + long time = end - start; + if (e instanceof EOFException) { + // NO STACKTRACE + log.error("Failed to send metrics, " + + "likely the client or this server closed " + + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); + } else { + log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + } + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + } finally { + long end = System.currentTimeMillis(); + long time = end - start; + try { + context.complete(); + } catch (IllegalStateException e) { + // this happens when metricsServletTimeoutMs expires + // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled + log.error("Failed to generate prometheus stats, " + + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + } + } } - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGeneratorUtils.generate(cluster, outputStream, metricsProviders); + private void generateMetricsSynchronously(HttpServletResponse res) throws IOException { + res.setStatus(HTTP_STATUS_OK_200); + res.setContentType("text/plain;charset=utf-8"); + PrometheusMetricsGeneratorUtils.generate(cluster, res.getOutputStream(), metricsProviders); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java deleted file mode 100644 index 08730189322ee..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.stats; - -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.function.Function; - -public final class TimeWindow { - private final int interval; - private final int sampleCount; - private final AtomicReferenceArray> array; - - public TimeWindow(int sampleCount, int interval) { - this.sampleCount = sampleCount; - this.interval = interval; - this.array = new AtomicReferenceArray<>(sampleCount); - } - - /** - * return current time window data. - * - * @param function generate data. - * @return - */ - public synchronized WindowWrap current(Function function) { - long millis = System.currentTimeMillis(); - - if (millis < 0) { - return null; - } - int idx = calculateTimeIdx(millis); - long windowStart = calculateWindowStart(millis); - while (true) { - WindowWrap old = array.get(idx); - if (old == null) { - WindowWrap window = new WindowWrap<>(interval, windowStart, null); - if (array.compareAndSet(idx, null, window)) { - T value = null == function ? null : function.apply(null); - window.value(value); - return window; - } else { - Thread.yield(); - } - } else if (windowStart == old.start()) { - return old; - } else if (windowStart > old.start()) { - T value = null == function ? null : function.apply(old.value()); - old.value(value); - old.resetWindowStart(windowStart); - return old; - } else { - //it should never goes here - throw new IllegalStateException(); - } - } - } - - private int calculateTimeIdx(long timeMillis) { - long timeId = timeMillis / this.interval; - return (int) (timeId % sampleCount); - } - - private long calculateWindowStart(long timeMillis) { - return timeMillis - timeMillis % this.interval; - } - - public int sampleCount() { - return sampleCount; - } - - public int interval() { - return interval; - } - - public long currentWindowStart(long millis) { - return this.calculateWindowStart(millis); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java deleted file mode 100644 index 12869b82921e5..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.stats; - -public final class WindowWrap { - private final long interval; - private long start; - private T value; - - public WindowWrap(long interval, long windowStart, T value) { - this.interval = interval; - this.start = windowStart; - this.value = value; - } - - public long interval() { - return this.interval; - } - - public long start() { - return this.start; - } - - public T value() { - return value; - } - - public void value(T value) { - this.value = value; - } - - public WindowWrap resetWindowStart(long startTime) { - this.start = startTime; - return this; - } - - public boolean isTimeInWindow(long timeMillis) { - return start <= timeMillis && timeMillis < start + interval; - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java new file mode 100644 index 0000000000000..b80e5747d8a5a --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.stats.prometheus; + +import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.hotspot.DefaultExports; +import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.common.util.DirectMemoryUtils; + +public class MetricsExports { + private static boolean initialized = false; + + private MetricsExports() { + } + + public static synchronized void initialize() { + if (!initialized) { + DefaultExports.initialize(); + register(CollectorRegistry.defaultRegistry); + initialized = true; + } + } + + public static void register(CollectorRegistry registry) { + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }).register(registry); + + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }).register(registry); + + // metric to export pulsar version info + Gauge.build("pulsar_version_info", "-") + .labelNames("version", "commit").create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return 1.0; + } + }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) + .register(registry); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 93cbad4e19503..5a5a61404b87f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -42,7 +42,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray stream.write(metricName).write('{'); for (int i = 0; i < labelsAndValuesArray.length; i += 2) { String labelValue = labelsAndValuesArray[i + 1]; - if (labelValue != null) { + if (labelValue != null && labelValue.indexOf('"') > -1) { labelValue = labelValue.replace("\"", "\\\""); } stream.write(labelsAndValuesArray[i]).write("=\"").write(labelValue).write('\"'); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 124f0d3e54e4f..bbd09335c0a97 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -20,40 +20,39 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.generateSystemMetrics; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.getTypeStr; -import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.Gauge; -import io.prometheus.client.Gauge.Child; -import io.prometheus.client.hotspot.DefaultExports; +import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.io.StringWriter; +import java.io.OutputStreamWriter; import java.io.Writer; -import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.TimeWindow; -import org.apache.pulsar.broker.stats.WindowWrap; import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.eclipse.jetty.server.HttpOutput; /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out @@ -62,123 +61,80 @@ * href="https://prometheus.io/docs/instrumenting/exposition_formats/">Exposition Formats */ @Slf4j -public class PrometheusMetricsGenerator { - private static volatile TimeWindow timeWindow; - private static final int MAX_COMPONENTS = 64; - - static { - DefaultExports.initialize(); - - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); - - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); - - // metric to export pulsar version info - Gauge.build("pulsar_version_info", "-") - .labelNames("version", "commit").create() - .setChild(new Child() { - @Override - public double get() { - return 1.0; - } - }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) - .register(CollectorRegistry.defaultRegistry); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, false, out, null); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, out, null); - } - - public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean includeProducerMetrics, - boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) throws IOException { - ByteBuf buffer; - boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); +public class PrometheusMetricsGenerator implements AutoCloseable { + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024; // 1MB + private static final int MINIMUM_FOR_MAX_COMPONENTS = 64; + + private volatile MetricsBuffer metricsBuffer; + private static AtomicReferenceFieldUpdater metricsBufferFieldUpdater = + AtomicReferenceFieldUpdater.newUpdater(PrometheusMetricsGenerator.class, MetricsBuffer.class, + "metricsBuffer"); + private volatile boolean closed; + + public static class MetricsBuffer { + private final CompletableFuture bufferFuture; + private final long createTimeslot; + private final AtomicInteger refCnt = new AtomicInteger(2); + + MetricsBuffer(long timeslot) { + bufferFuture = new CompletableFuture<>(); + createTimeslot = timeslot; + } - if (!exposeBufferMetrics) { - buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - } else { - if (null == timeWindow) { - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); - } - WindowWrap window = timeWindow.current(oldBuf -> { - // release expired buffer, in case of memory leak - if (oldBuf != null && oldBuf.refCnt() > 0) { - oldBuf.release(); - log.debug("Cached metrics buffer released"); - } + public CompletableFuture getBufferFuture() { + return bufferFuture; + } - try { - ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - log.debug("Generated metrics buffer size {}", buf.readableBytes()); - return buf; - } catch (IOException e) { - log.error("Generate metrics failed", e); - //return empty buffer if exception happens - return PulsarByteBufAllocator.DEFAULT.heapBuffer(0); - } - }); + long getCreateTimeslot() { + return createTimeslot; + } - if (null == window || null == window.value()) { - return; - } - buffer = window.value(); - log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); + /** + * Retain the buffer. This is allowed, only when the buffer is not already released. + * + * @return true if the buffer is retained successfully, false otherwise. + */ + boolean retain() { + return refCnt.updateAndGet(x -> x > 0 ? x + 1 : x) > 0; } - try { - if (out instanceof HttpOutput) { - HttpOutput output = (HttpOutput) out; - //no mem_copy and memory allocations here - ByteBuffer[] buffers = buffer.nioBuffers(); - for (ByteBuffer buffer0 : buffers) { - output.write(buffer0); - } - } else { - //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. - //not modify buffer readIndex/writeIndex here. - int readIndex = buffer.readerIndex(); - int readableBytes = buffer.readableBytes(); - for (int i = 0; i < readableBytes; i++) { - out.write(buffer.getByte(readIndex + i)); - } - } - } finally { - if (!exposeBufferMetrics && buffer.refCnt() > 0) { - buffer.release(); - log.debug("Metrics buffer released."); + /** + * Release the buffer. + */ + public void release() { + int newValue = refCnt.decrementAndGet(); + if (newValue == 0) { + bufferFuture.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); } } } - private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - List metricsProviders) throws IOException { - //Use unpooled buffers here to avoid direct buffer usage increasing. - //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + private final PulsarService pulsar; + private final boolean includeTopicMetrics; + private final boolean includeConsumerMetrics; + private final boolean includeProducerMetrics; + private final boolean splitTopicAndPartitionIndexLabel; + private final Clock clock; + + private volatile int initialBufferSize = DEFAULT_INITIAL_BUFFER_SIZE; + + public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, + boolean splitTopicAndPartitionIndexLabel, Clock clock) { + this.pulsar = pulsar; + this.includeTopicMetrics = includeTopicMetrics; + this.includeConsumerMetrics = includeConsumerMetrics; + this.includeProducerMetrics = includeProducerMetrics; + this.splitTopicAndPartitionIndexLabel = splitTopicAndPartitionIndexLabel; + this.clock = clock; + } + + private ByteBuf generate0(List metricsProviders) { + ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); @@ -220,10 +176,34 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri //if exception happens, release buffer if (exceptionHappens) { buf.release(); + } else { + // for the next time, the initial buffer size will be suggested by the last buffer size + initialBufferSize = Math.max(DEFAULT_INITIAL_BUFFER_SIZE, buf.readableBytes()); } } } + private ByteBuf allocateMultipartCompositeDirectBuffer() { + // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used + // for allocating the buffers + ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( + Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); + int totalLen = 0; + while (totalLen < initialBufferSize) { + totalLen += chunkSize; + buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + } + return buf; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -269,12 +249,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = key.substring(0, nameIndex); value = key.substring(nameIndex + 1); if (!names.contains(name)) { - stream.write("# TYPE ").write(name.replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write("\n"); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, name); + stream.write(' ').write(getTypeStr(metricType)).write("\n"); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } catch (Exception e) { continue; } @@ -283,12 +264,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = entry.getKey(); if (!names.contains(name)) { - stream.write("# TYPE ").write(entry.getKey().replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write('\n'); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, entry.getKey()); + stream.write(' ').write(getTypeStr(metricType)).write('\n'); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } //to avoid quantile label duplicated @@ -308,18 +290,98 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, } } + private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutputStream stream, String name) { + if (name.startsWith("brk_")) { + return stream.write("pulsar_").write(CharBuffer.wrap(name).position("brk_".length())); + } else { + return stream.write(name); + } + } + private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); if (statsProvider instanceof NullStatsProvider) { return; } - try { - Writer writer = new StringWriter(); + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + @Override + public void write(int b) throws IOException { + stream.writeByte(b); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + stream.write(b, off, len); + } + }), StandardCharsets.UTF_8)) { statsProvider.writeAllMetrics(writer); - stream.write(writer.toString()); } catch (IOException e) { - // nop + log.error("Failed to write managed ledger bookie client metrics", e); + } + } + + public MetricsBuffer renderToBuffer(Executor executor, List metricsProviders) { + boolean cacheMetricsResponse = pulsar.getConfiguration().isMetricsBufferResponse(); + while (!closed && !Thread.currentThread().isInterrupted()) { + long currentTimeSlot = cacheMetricsResponse ? calculateCurrentTimeSlot() : 0; + MetricsBuffer currentMetricsBuffer = metricsBuffer; + if (currentMetricsBuffer == null || currentMetricsBuffer.getBufferFuture().isCompletedExceptionally() + || (currentMetricsBuffer.getBufferFuture().isDone() + && (currentMetricsBuffer.getCreateTimeslot() != 0 + && currentTimeSlot > currentMetricsBuffer.getCreateTimeslot()))) { + MetricsBuffer newMetricsBuffer = new MetricsBuffer(currentTimeSlot); + if (metricsBufferFieldUpdater.compareAndSet(this, currentMetricsBuffer, newMetricsBuffer)) { + if (currentMetricsBuffer != null) { + currentMetricsBuffer.release(); + } + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); + executor.execute(() -> { + try { + bufferFuture.complete(generate0(metricsProviders)); + } catch (Exception e) { + bufferFuture.completeExceptionally(e); + } finally { + if (currentTimeSlot == 0) { + // if the buffer is not cached, release it after the future is completed + metricsBufferFieldUpdater.compareAndSet(this, newMetricsBuffer, null); + newMetricsBuffer.release(); + } + } + }); + // no need to retain before returning since the new buffer starts with refCnt 2 + return newMetricsBuffer; + } else { + currentMetricsBuffer = metricsBuffer; + } + } + // retain the buffer before returning + // if the buffer is already released, retaining won't succeed, retry in that case + if (currentMetricsBuffer != null && currentMetricsBuffer.retain()) { + return currentMetricsBuffer; + } + } + return null; + } + + /** + * Calculate the current time slot based on the current time. + * This is to ensure that cached metrics are refreshed consistently at a fixed interval regardless of the request + * time. + */ + private long calculateCurrentTimeSlot() { + long cacheTimeoutMillis = + TimeUnit.SECONDS.toMillis(Math.max(1, pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds())); + long now = clock.millis(); + return now / cacheTimeoutMillis; + } + + @Override + public void close() { + closed = true; + MetricsBuffer buffer = metricsBufferFieldUpdater.getAndSet(this, null); + if (buffer != null) { + buffer.release(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 42bd2652883b6..7fcc74e965c24 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,34 +18,142 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; +import org.eclipse.jetty.server.HttpOutput; +@Slf4j public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { - private static final long serialVersionUID = 1L; + private static final int EXECUTOR_MAX_THREADS = 4; - private final PulsarService pulsar; - private final boolean shouldExportTopicMetrics; - private final boolean shouldExportConsumerMetrics; - private final boolean shouldExportProducerMetrics; - private final boolean splitTopicAndPartitionLabel; + private final PrometheusMetricsGenerator prometheusMetricsGenerator; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean shouldExportProducerMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean splitTopicAndPartitionLabel) { - super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName()); - this.pulsar = pulsar; - this.shouldExportTopicMetrics = includeTopicMetrics; - this.shouldExportConsumerMetrics = includeConsumerMetrics; - this.shouldExportProducerMetrics = shouldExportProducerMetrics; - this.splitTopicAndPartitionLabel = splitTopicAndPartitionLabel; + super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName(), + EXECUTOR_MAX_THREADS); + MetricsExports.initialize(); + prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); } + @Override - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGenerator.generate(pulsar, shouldExportTopicMetrics, shouldExportConsumerMetrics, - shouldExportProducerMetrics, splitTopicAndPartitionLabel, outputStream, metricsProviders); + public void destroy() { + super.destroy(); + prometheusMetricsGenerator.close(); + } + + protected void doGet(HttpServletRequest request, HttpServletResponse response) { + AsyncContext context = request.startAsync(); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean skipWritingResponse = new AtomicBoolean(false); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent event) throws IOException { + } + + @Override + public void onTimeout(AsyncEvent event) throws IOException { + log.warn("Prometheus metrics request timed out"); + skipWritingResponse.set(true); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent event) throws IOException { + skipWritingResponse.set(true); + } + + @Override + public void onStartAsync(AsyncEvent event) throws IOException { + } + }); + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + prometheusMetricsGenerator.renderToBuffer(executor, metricsProviders); + if (metricsBuffer == null) { + log.info("Service is closing, skip writing metrics."); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + context.complete(); + return; + } + metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + try { + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos( + metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted() && !skipWritingResponse.get()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + return; + } + if (skipWritingResponse.get()) { + log.warn("Response has timed or failed, skip writing metrics."); + return; + } + if (response.isCommitted()) { + log.warn("Response is already committed, cannot write metrics"); + return; + } + if (ex != null) { + log.error("Failed to generate metrics", ex); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + return; + } + if (buffer == null) { + log.error("Failed to generate metrics, buffer is null"); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } else { + response.setStatus(HTTP_STATUS_OK_200); + response.setContentType("text/plain;charset=utf-8"); + ServletOutputStream outputStream = response.getOutputStream(); + if (outputStream instanceof HttpOutput) { + HttpOutput output = (HttpOutput) outputStream; + for (ByteBuffer nioBuffer : buffer.nioBuffers()) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(outputStream, length); + } + } + } + } catch (EOFException e) { + log.error("Failed to write metrics to response due to EOFException"); + } catch (IOException e) { + log.error("Failed to write metrics to response", e); + } finally { + metricsBuffer.release(); + context.complete(); + } + })); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 4be006423f509..27288291d2969 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -507,7 +507,9 @@ private static void writeConsumerMetric(PrometheusMetricStreams stream, String m static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, String... extraLabelsAndValues) { - String[] labelsAndValues = new String[splitTopicAndPartitionIndexLabel ? 8 : 6]; + int baseLabelCount = splitTopicAndPartitionIndexLabel ? 8 : 6; + String[] labelsAndValues = + new String[baseLabelCount + (extraLabelsAndValues != null ? extraLabelsAndValues.length : 0)]; labelsAndValues[0] = "cluster"; labelsAndValues[1] = cluster; labelsAndValues[2] = "namespace"; @@ -527,7 +529,11 @@ static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, } else { labelsAndValues[5] = topic; } - String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); - stream.writeSample(metricName, value, labels); + if (extraLabelsAndValues != null) { + for (int i = 0; i < extraLabelsAndValues.length; i++) { + labelsAndValues[baseLabelCount + i] = extraLabelsAndValues[i]; + } + } + stream.writeSample(metricName, value, labelsAndValues); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java new file mode 100644 index 0000000000000..fcc3b6aa88fb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; +import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; +import org.eclipse.jetty.server.HttpOutput; + +public class PrometheusMetricsTestUtil { + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, false, Clock.systemUTC()), out, null); + } + + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionIndexLabel, Clock.systemUTC()), out, null); + } + + public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputStream out, + List metricsProviders) throws IOException { + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + metricsGenerator.renderToBuffer(MoreExecutors.directExecutor(), metricsProviders); + try { + ByteBuf buffer = null; + try { + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + if (buffer == null) { + return; + } + if (out instanceof HttpOutput) { + HttpOutput output = (HttpOutput) out; + ByteBuffer[] nioBuffers = buffer.nioBuffers(); + for (ByteBuffer nioBuffer : nioBuffers) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(out, length); + } + } + } finally { + metricsBuffer.release(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 8be0aa4bc7dbd..ff8e418c024a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -40,10 +40,10 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -218,7 +218,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { Thread.sleep(2000); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricsStr); @@ -304,7 +304,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { assertEquals(opLatencyMetricsSum.intValue(), opLatencyTopicMetrics.get().value); ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); + PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); Multimap namespaceMetricsMap = parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); Optional namespaceMetric = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index c214634e6ed32..44d24668cc381 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -65,11 +65,11 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -367,7 +367,7 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex latch.await(10, TimeUnit.SECONDS); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index a520b8c241bd1..3a4016eb79c21 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -43,11 +43,11 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; @@ -121,7 +121,7 @@ public void testSchemaRegistryMetrics() throws Exception { deleteSchema(schemaId, version(1)); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index eb4500c13667a..512a5cfcab661 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -45,6 +45,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; @@ -52,7 +53,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -335,7 +335,7 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel consumer2.updateRates(); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 15f41365da8d1..726bde3f3d0a9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -30,10 +30,10 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -101,7 +101,7 @@ public void testMetadataStoreStats() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = parseMetrics(metricsStr); @@ -191,7 +191,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index d3891931496c5..1fe0e99b49874 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -21,7 +21,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -35,6 +38,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -51,6 +55,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -61,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -85,7 +91,6 @@ import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.zookeeper.CreateMode; import org.awaitility.Awaitility; -import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -155,7 +160,7 @@ public void testPublishRateLimitedTimes() throws Exception { }); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); @@ -185,7 +190,7 @@ public void testPublishRateLimitedTimes() throws Exception { @Cleanup ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); @@ -217,7 +222,7 @@ public void testMetricsTopicCount() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_topics_count"); @@ -254,7 +259,7 @@ public void testMetricsAvgMsgSize2() throws Exception { producerInServer.getStats().msgThroughputIn = 100; @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); @@ -297,7 +302,7 @@ public void testPerTopicStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -395,7 +400,7 @@ public void testPerBrokerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -504,7 +509,7 @@ public void testPerTopicStatsReconnect() throws Exception { c2.close(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -582,7 +587,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric true ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -614,7 +619,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric false ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); @@ -698,7 +703,7 @@ public void testPerTopicExpiredStat() throws Exception { Awaitility.await().until(() -> sub2.getExpiredMessageRate() != 0.0); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic @@ -780,15 +785,15 @@ public void testBundlesMetrics() throws Exception { for (var latencyMetric : UnloadManager.LatencyMetric.values()) { var serviceUnit = "serviceUnit"; var brokerLookupAddress = "lookupAddress"; - var serviceUnitStateData = Mockito.mock(ServiceUnitStateData.class); - Mockito.when(serviceUnitStateData.sourceBroker()).thenReturn(brokerLookupAddress); - Mockito.when(serviceUnitStateData.dstBroker()).thenReturn(brokerLookupAddress); + var serviceUnitStateData = mock(ServiceUnitStateData.class); + when(serviceUnitStateData.sourceBroker()).thenReturn(brokerLookupAddress); + when(serviceUnitStateData.dstBroker()).thenReturn(brokerLookupAddress); latencyMetric.beginMeasurement(serviceUnit, brokerLookupAddress, serviceUnitStateData); latencyMetric.endMeasurement(serviceUnit); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); @@ -838,7 +843,7 @@ public void testNonPersistentSubMetrics() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); @@ -885,7 +890,7 @@ public void testPerNamespaceStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -958,7 +963,7 @@ public void testPerProducerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1026,7 +1031,7 @@ public void testPerConsumerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1113,7 +1118,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); @@ -1217,7 +1222,7 @@ public void testManagedLedgerCacheStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1253,7 +1258,7 @@ public void testManagedLedgerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1331,7 +1336,7 @@ public void testManagedLedgerBookieClientStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1412,7 +1417,7 @@ public String getCommandData() { }); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_authentication_success_total"); @@ -1473,7 +1478,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_expired_token_total"); @@ -1514,7 +1519,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); @@ -1588,7 +1593,7 @@ public void testManagedCursorPersistStats() throws Exception { // enable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(true); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1601,7 +1606,7 @@ public void testManagedCursorPersistStats() throws Exception { // disable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(false); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); @@ -1620,7 +1625,7 @@ public void testBrokerConnection() throws Exception { .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_connection_created_total_count"); @@ -1637,7 +1642,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1660,7 +1665,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1704,7 +1709,7 @@ public void testCompaction() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1739,7 +1744,7 @@ public void testCompaction() throws Exception { Compactor compactor = ((PulsarCompactionServiceFactory)pulsar.getCompactionServiceFactory()).getCompactor(); compactor.compact(topicName).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1772,31 +1777,36 @@ public void testCompaction() throws Exception { @Test public void testMetricsWithCache() throws Throwable { - ServiceConfiguration configuration = Mockito.mock(ServiceConfiguration.class); - Mockito.when(configuration.getManagedLedgerStatsPeriodSeconds()).thenReturn(2); - Mockito.when(configuration.isMetricsBufferResponse()).thenReturn(true); - Mockito.when(configuration.getClusterName()).thenReturn(configClusterName); - Mockito.when(pulsar.getConfiguration()).thenReturn(configuration); + ServiceConfiguration configuration = pulsar.getConfiguration(); + configuration.setManagedLedgerStatsPeriodSeconds(2); + configuration.setMetricsBufferResponse(true); + configuration.setClusterName(configClusterName); - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - TimeWindow timeWindow = new TimeWindow<>(2, (int) TimeUnit.SECONDS.toMillis(period)); + // create a mock clock to control the time + AtomicLong currentTimeMillis = new AtomicLong(System.currentTimeMillis()); + Clock clock = mock(); + when(clock.millis()).thenAnswer(invocation -> currentTimeMillis.get()); + PrometheusMetricsGenerator prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, true, false, false, + false, clock); + + String previousMetrics = null; for (int a = 0; a < 4; a++) { - long start = System.currentTimeMillis(); ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut1, null); + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut1, null); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut2, null); - long end = System.currentTimeMillis(); - - if (timeWindow.currentWindowStart(start) == timeWindow.currentWindowStart(end)) { - String metricsStr1 = statsOut1.toString(); - String metricsStr2 = statsOut2.toString(); - assertEquals(metricsStr1, metricsStr2); - Multimap metrics = parseMetrics(metricsStr1); - } + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut2, null); + + String metricsStr1 = statsOut1.toString(); + String metricsStr2 = statsOut2.toString(); + assertTrue(metricsStr1.length() > 1000); + assertEquals(metricsStr1, metricsStr2); + assertNotEquals(metricsStr1, previousMetrics); + previousMetrics = metricsStr1; - Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); + // move time forward + currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } } @@ -1824,7 +1834,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_consumers_count"); @@ -1860,7 +1870,7 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Pattern typePattern = Pattern.compile("^#\\s+TYPE\\s+(\\w+)\\s+(\\w+)"); @@ -1920,7 +1930,7 @@ public void testEscapeLabelValue() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); final List subCountLines = metricsStr.lines() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index e39860274d12f..3e71d8f211101 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -31,13 +31,13 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -233,7 +233,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java deleted file mode 100644 index 89528c1965397..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.broker.stats; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import org.testng.annotations.Test; - -public class TimeWindowTest { - - @Test - public void windowTest() throws Exception { - int intervalInMs = 1000; - int sampleCount = 2; - TimeWindow timeWindow = new TimeWindow<>(sampleCount, intervalInMs); - - WindowWrap expect1 = timeWindow.current(oldValue -> 1); - WindowWrap expect2 = timeWindow.current(oldValue -> null); - assertNotNull(expect1); - assertNotNull(expect2); - - if (expect1.start() == expect2.start()) { - assertEquals((int) expect1.value(), 1); - assertEquals(expect1, expect2); - assertEquals(expect1.value(), expect2.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect3 = timeWindow.current(oldValue -> 2); - WindowWrap expect4 = timeWindow.current(oldValue -> null); - assertNotNull(expect3); - assertNotNull(expect4); - - if (expect3.start() == expect4.start()) { - assertEquals((int) expect3.value(), 2); - assertEquals(expect3, expect4); - assertEquals(expect3.value(), expect4.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect5 = timeWindow.current(oldValue -> 3); - WindowWrap expect6 = timeWindow.current(oldValue -> null); - assertNotNull(expect5); - assertNotNull(expect6); - - if (expect5.start() == expect6.start()) { - assertEquals((int) expect5.value(), 3); - assertEquals(expect5, expect6); - assertEquals(expect5.value(), expect6.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect7 = timeWindow.current(oldValue -> 4); - WindowWrap expect8 = timeWindow.current(oldValue -> null); - assertNotNull(expect7); - assertNotNull(expect8); - - if (expect7.start() == expect8.start()) { - assertEquals((int) expect7.value(), 4); - assertEquals(expect7, expect8); - assertEquals(expect7.value(), expect8.value()); - } - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 723a493eca1df..8d5cb9dc39148 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -38,9 +38,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -118,7 +118,7 @@ public void testTransactionCoordinatorMetrics() throws Exception { pulsar.getTransactionMetadataStoreService().getStores() .get(transactionCoordinatorIDTwo).newTransaction(timeout, null).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_txn_active_count"); @@ -186,7 +186,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { pulsar.getBrokerService().updateRates(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -216,7 +216,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { }); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -272,7 +272,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -290,7 +290,7 @@ public void testManagedLedgerMetrics() throws Exception { checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 126, metric); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -334,7 +334,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -359,7 +359,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -393,7 +393,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { .send(); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index be036a0cf590b..1c3de777e9349 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -46,9 +46,9 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; @@ -229,7 +229,7 @@ public void testTransactionBufferMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 6c24b6b3f0151..db9daf56104c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -46,12 +46,12 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -255,7 +255,7 @@ public void testPendingAckMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index d2b59ed0e4997..17588a7ecac8b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -55,9 +55,9 @@ import javax.net.ssl.TrustManager; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -107,7 +107,7 @@ public class WebServiceTest { public void testWebExecutorMetrics() throws Exception { setupEnv(true, false, false, false, -1, false); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index c8c639606aa3e..9bf6302f50f02 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -20,6 +20,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.CharsetUtil; +import java.nio.CharBuffer; /** * Format strings and numbers into a ByteBuf without any memory allocation. @@ -28,6 +29,7 @@ public class SimpleTextOutputStream { private final ByteBuf buffer; private static final char[] hexChars = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; + private final CharBuffer singleCharBuffer = CharBuffer.allocate(1); public SimpleTextOutputStream(ByteBuf buffer) { this.buffer = buffer; @@ -44,11 +46,17 @@ public SimpleTextOutputStream write(byte[] a, int offset, int len) { } public SimpleTextOutputStream write(char c) { - write(String.valueOf(c)); + // In UTF-8, any character from U+0000 to U+007F is encoded in one byte + if (c <= '\u007F') { + buffer.writeByte((byte) c); + return this; + } + singleCharBuffer.put(0, c); + buffer.writeCharSequence(singleCharBuffer, CharsetUtil.UTF_8); return this; } - public SimpleTextOutputStream write(String s) { + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; } @@ -136,4 +144,8 @@ public void write(ByteBuf byteBuf) { public ByteBuf getBuffer() { return buffer; } + + public void writeByte(int b) { + buffer.writeByte(b); + } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index db2969e3c3920..39c8fb5e086fd 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 @@ -392,6 +392,12 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private boolean authenticateMetricsEndpoint = true; + @FieldContext( + category = CATEGORY_HTTP, + doc = "Time in milliseconds that metrics endpoint would time out. Default is 30s.\n" + + " Set it to 0 to disable timeout." + ) + private long metricsServletTimeoutMs = 30000; @FieldContext( category = CATEGORY_SASL_AUTH, diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index 61b00871cecdb..ea9e4ebfaa9b8 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -299,7 +299,8 @@ public void start() throws Exception { } private synchronized void createMetricsServlet() { - this.metricsServlet = new PrometheusMetricsServlet(-1L, proxyConfig.getClusterName()); + this.metricsServlet = + new PrometheusMetricsServlet(proxyConfig.getMetricsServletTimeoutMs(), proxyConfig.getClusterName()); if (pendingMetricsProviders != null) { pendingMetricsProviders.forEach(provider -> metricsServlet.addRawMetricsProvider(provider)); this.pendingMetricsProviders = null; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 72d54601995f1..50a8e3ab7d753 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -24,6 +24,7 @@ import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import com.google.common.annotations.VisibleForTesting; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; @@ -236,21 +237,36 @@ public void start() throws Exception { if (!metricsInitialized) { // Setup metrics DefaultExports.initialize(); + CollectorRegistry registry = CollectorRegistry.defaultRegistry; // Report direct memory from Netty counters - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesUsed = + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }); + try { + registry.register(jvmMemoryDirectBytesUsed); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_used metric: {}", e.getMessage()); + } - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesMax = + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }); + try { + registry.register(jvmMemoryDirectBytesMax); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_max metric: {}", e.getMessage()); + } metricsInitialized = true; } From 837f8bca7ddbbad4354f9a89e36fcd6aea1be85c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Apr 2024 00:13:49 +0800 Subject: [PATCH 21/55] [fix] [broker] Prevent long deduplication cursor backlog so that topic loading wouldn't timeout (#22479) --- .../pulsar/broker/service/BrokerService.java | 4 +- .../persistent/MessageDeduplication.java | 18 +- .../service/persistent/PersistentTopic.java | 2 +- .../DeduplicationDisabledBrokerLevelTest.java | 161 ++++++++++++++++++ 4 files changed, 177 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index b4d0f38b4a4dc..2687532693a45 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -631,8 +631,10 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd } protected void startDeduplicationSnapshotMonitor() { + // We do not know whether users will enable deduplication on namespace level/topic level or not, so keep this + // scheduled task runs. int interval = pulsar().getConfiguration().getBrokerDeduplicationSnapshotFrequencyInSeconds(); - if (interval > 0 && pulsar().getConfiguration().isBrokerDeduplicationEnabled()) { + if (interval > 0) { this.deduplicationSnapshotMonitor = OrderedScheduler.newSchedulerBuilder() .name("deduplication-snapshot-monitor") .numThreads(1) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index 802dd91796127..e508661364d74 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -157,9 +157,14 @@ private CompletableFuture recoverSequenceIdsMap() { // Replay all the entries and apply all the sequence ids updates log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future; + return future.thenAccept(lastPosition -> { + if (lastPosition != null && snapshotCounter >= snapshotInterval) { + snapshotCounter = 0; + takeSnapshot(lastPosition); + } + }); } /** @@ -168,11 +173,11 @@ private CompletableFuture recoverSequenceIdsMap() { * * @param future future to trigger when the replay is complete */ - private void replayCursor(CompletableFuture future) { + private void replayCursor(CompletableFuture future) { managedCursor.asyncReadEntries(100, new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { - + Position lastPosition = null; for (Entry entry : entries) { ByteBuf messageMetadataAndPayload = entry.getDataBuffer(); MessageMetadata md = Commands.parseMessageMetadata(messageMetadataAndPayload); @@ -182,7 +187,8 @@ public void readEntriesComplete(List entries, Object ctx) { highestSequencedPushed.put(producerName, sequenceId); highestSequencedPersisted.put(producerName, sequenceId); producerRemoved(producerName); - + snapshotCounter++; + lastPosition = entry.getPosition(); entry.release(); } @@ -191,7 +197,7 @@ public void readEntriesComplete(List entries, Object ctx) { pulsar.getExecutor().execute(() -> replayCursor(future)); } else { // Done replaying - future.complete(null); + future.complete(lastPosition); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3c9ab04d79a0d..e4441969101c1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -208,7 +208,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal private volatile List shadowTopics; private final TopicName shadowSourceTopic; - static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; + public static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; public static boolean isDedupCursorName(String name) { return DEDUPLICATION_CURSOR_NAME.equals(name); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java new file mode 100644 index 0000000000000..2ce4ea9b00b2e --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.MessageDeduplication; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class DeduplicationDisabledBrokerLevelTest extends ProducerConsumerBase { + + private int deduplicationSnapshotFrequency = 5; + private int brokerDeduplicationEntriesInterval = 1000; + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + this.conf.setBrokerDeduplicationEnabled(false); + this.conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(deduplicationSnapshotFrequency); + this.conf.setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + } + + @Test + public void testNoBacklogOnDeduplication() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + // deduplication enabled: + // broker level: "false" + // topic level: "true". + // So it is enabled. + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + + // Verify: regarding deduplication cursor, messages will be acknowledged automatically. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + producer.close(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + Awaitility.await().atMost(Duration.ofSeconds(deduplicationSnapshotFrequency * 3)).untilAsserted(() -> { + PositionImpl LAC = (PositionImpl) ml.getLastConfirmedEntry(); + PositionImpl cursorMD = (PositionImpl) cursor.getMarkDeletedPosition(); + assertTrue(LAC.compareTo(cursorMD) <= 0); + }); + + // cleanup. + admin.topics().delete(topic); + } + + @Test + public void testSnapshotCounterAfterUnload() throws Exception { + final int originalDeduplicationSnapshotFrequency = deduplicationSnapshotFrequency; + deduplicationSnapshotFrequency = 3600; + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + // 1. Send 999 messages, it is less than "brokerDeduplicationEntriesIntervaddl". + // 2. Unload topic. + // 3. Send 1 messages, there are 1099 messages have not been snapshot now. + // 4. Verify the snapshot has been taken. + // step 1. + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + admin.topics().unload(topic); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml2.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + // step 3. + producer.send("last message"); + ml2.trimConsumedLedgersInBackground(new CompletableFuture<>()); + // step 4. + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + assertTrue(snapshotCounter3 < brokerDeduplicationEntriesInterval); + // Verify: the previous ledger will be removed because all messages have been acked. + assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + // cleanup. + producer.close(); + admin.topics().delete(topic); + deduplicationSnapshotFrequency = originalDeduplicationSnapshotFrequency; + cleanup(); + setup(); + } +} From 5d18ff7b70f9de3b95d83f6a8fd4756b1c34567b Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 15 Apr 2024 08:43:12 +0800 Subject: [PATCH 22/55] [fix][txn]Handle exceptions in the transaction pending ack init (#21274) Co-authored-by: Baodi Shi --- .../pulsar/broker/service/ServerCnx.java | 2 +- .../pendingack/impl/PendingAckHandleImpl.java | 54 ++++++++++-- .../broker/transaction/TransactionTest.java | 2 +- .../pendingack/PendingAckPersistentTest.java | 82 +++++++++++++++++++ 4 files changed, 132 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 4ee6ac43465f4..a60f1d805ceb6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1376,7 +1376,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { // Send error back to client, only if not completed already. if (consumerFuture.completeExceptionally(exception)) { commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(exception), + BrokerServiceException.getClientErrorCode(exception.getCause()), exception.getCause().getMessage()); } consumers.remove(consumerId, consumerFuture); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 7dbe0385fd7e9..5ed271c6fd414 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; import com.google.common.annotations.VisibleForTesting; +import io.netty.util.Timer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,9 +36,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -45,6 +48,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.Consumer; @@ -53,7 +57,9 @@ import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleStats; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; @@ -134,6 +140,12 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi public final RecoverTimeRecord recoverTime = new RecoverTimeRecord(); + private final long pendingAckInitFailureBackoffInitialTimeInMs = 100; + + public final Backoff backoff = new Backoff(pendingAckInitFailureBackoffInitialTimeInMs, TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + + private final Timer transactionOpTimer; public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { super(State.None); @@ -153,7 +165,11 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { this.pendingAckStoreProvider = this.persistentSubscription.getTopic() .getBrokerService().getPulsar().getTransactionPendingAckStoreProvider(); + transactionOpTimer = persistentSubscription.getTopic().getBrokerService().getPulsar().getTransactionTimer(); + init(); + } + private void init() { pendingAckStoreProvider.checkInitializedBefore(persistentSubscription) .thenAcceptAsync(init -> { if (init) { @@ -164,9 +180,9 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { }, internalPinnedExecutor) .exceptionallyAsync(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - changeToErrorState(); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(t); - this.pendingAckStoreFuture.completeExceptionally(t); return null; }, internalPinnedExecutor); } @@ -180,9 +196,8 @@ private void initPendingAckStore() { recoverTime.setRecoverStartTime(System.currentTimeMillis()); pendingAckStore.replayAsync(this, internalPinnedExecutor); }).exceptionallyAsync(e -> { - handleCacheRequest(); - changeToErrorState(); - log.error("PendingAckHandleImpl init fail! TopicName : {}, SubName: {}", topicName, subName, e); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(e.getCause()); return null; }, internalPinnedExecutor); @@ -945,12 +960,39 @@ public void completeHandleFuture() { } public void exceptionHandleFuture(Throwable t) { - final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally(t); + if (isRetryableException(t)) { + this.state = State.None; + long retryTime = backoff.next(); + log.warn("[{}][{}] Failed to init transaction pending ack. It will be retried in {} Ms", + persistentSubscription.getTopic().getName(), subName, retryTime, t); + transactionOpTimer.newTimeout((timeout) -> init(), retryTime, TimeUnit.MILLISECONDS); + return; + } + log.error("[{}] [{}] PendingAckHandleImpl init fail!", topicName, subName, t); + handleCacheRequest(); + changeToErrorState(); + // ToDo: Add a new serverError `TransactionComponentLoadFailedException` + // and before that a `Unknown` will be returned first. + this.pendingAckStoreFuture = FutureUtil.failedFuture(new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); + final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally( + new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); if (completedNow) { recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } + private static boolean isRetryableException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return (realCause instanceof ManagedLedgerException + && !(realCause instanceof ManagedLedgerException.ManagedLedgerFencedException) + && !(realCause instanceof ManagedLedgerException.NonRecoverableLedgerException)) + || realCause instanceof PulsarClientException.BrokerPersistenceException + || realCause instanceof PulsarClientException.LookupException + || realCause instanceof PulsarClientException.ConnectException; + } + @Override public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID) { TransactionInPendingAckStats transactionInPendingAckStats = new TransactionInPendingAckStats(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index ddfa82f52886f..e45924e8bb4f2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1517,7 +1517,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { fail("Expect failure by PendingAckHandle closed, but success"); } catch (ExecutionException executionException){ Throwable t = executionException.getCause(); - Assert.assertTrue(t instanceof BrokerServiceException.ServiceUnitNotReadyException); + Assert.assertTrue(t instanceof BrokerServiceException); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index db9daf56104c9..93a2f274517d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -23,7 +23,9 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; @@ -44,9 +46,11 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -59,6 +63,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -100,6 +105,83 @@ protected void cleanup() { super.internalCleanup(); } + /** + * Test consumer can be built successfully with retryable exception + * and get correct error with no-retryable exception. + * @throws Exception + */ + @Test(timeOut = 60000) + public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { + // 1. Prepare and make sure the consumer can be built successfully. + String topic = NAMESPACE1 + "/testUnloadSubscriptionWhenFailedInitPendingAck"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .subscriptionName("subName1") + .topic(topic) + .subscribe(); + // 2. Mock a transactionPendingAckStoreProvider to test building consumer + // failing at transactionPendingAckStoreProvider::checkInitializedBefore. + Field transactionPendingAckStoreProviderField = PulsarService.class + .getDeclaredField("transactionPendingAckStoreProvider"); + transactionPendingAckStoreProviderField.setAccessible(true); + TransactionPendingAckStoreProvider pendingAckStoreProvider = + (TransactionPendingAckStoreProvider) transactionPendingAckStoreProviderField + .get(pulsarServiceList.get(0)); + TransactionPendingAckStoreProvider mockProvider = mock(pendingAckStoreProvider.getClass()); + // 3. Test retryable exception when checkInitializedBefore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + // First, the method checkInitializedBefore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail initialize"))) + // Then, the method will be executed successfully. + .thenReturn(CompletableFuture.completedFuture(false)); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer() + .subscriptionName("subName2") + .topic(topic) + .subscribe(); + + // 4. Test retryable exception when newPendingAckStore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + .thenReturn(CompletableFuture.completedFuture(true)); + + when(mockProvider.newPendingAckStore(any())) + // First, the method newPendingAckStore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail new store"))) + // Then, the method will be executed successfully. + .thenCallRealMethod(); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer3 = pulsarClient.newConsumer() + .subscriptionName("subName3") + .topic(topic) + .subscribe(); + + // 5. Test no-retryable exception: + // The consumer building will be failed without retrying. + when(mockProvider.checkInitializedBefore(any())) + // The method checkInitializedBefore will fail with a no-retryable exception without retrying. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException + .NonRecoverableLedgerException("mock fail"))) + .thenReturn(CompletableFuture.completedFuture(false)); + @Cleanup PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .build(); + try { + @Cleanup + Consumer consumer4 = pulsarClient.newConsumer() + .subscriptionName("subName4") + .topic(topic) + .subscribe(); + fail(); + } catch (Exception exception) { + assertTrue(exception.getMessage().contains("Failed to init transaction pending ack.")); + } + } + @Test public void individualPendingAckReplayTest() throws Exception { int messageCount = 1000; From d9a43dd21605930e16bb038095e36fceff3a4a40 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 15 Apr 2024 13:55:34 +0800 Subject: [PATCH 23/55] [fix][test] Flaky-test: testMessageExpiryWithTimestampNonRecoverableException and testIncorrectClientClock (#22489) --- .../service/PersistentMessageFinderTest.java | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 6965ac28068c1..0972c9098b55b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; @@ -383,7 +382,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test(groups = "flaky") + @Test void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -402,11 +401,15 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { for (int i = 0; i < totalEntries; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i)); } + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); List ledgers = ledger.getLedgersInfoAsList(); LedgerInfo lastLedgerInfo = ledgers.get(ledgers.size() - 1); - - assertEquals(ledgers.size(), totalEntries / entriesPerLedger); + // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(lastLedgerInfo.getEntries(), 0); + assertEquals(ledgers.size(), totalEntries / entriesPerLedger + 1); // this will make sure that all entries should be deleted Thread.sleep(TimeUnit.SECONDS.toMillis(ttlSeconds)); @@ -420,19 +423,13 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); - Position previousMarkDelete = null; - for (int i = 0; i < totalEntries; i++) { - monitor.expireMessages(1); - Position previousPos = previousMarkDelete; - retryStrategically( - (test) -> c1.getMarkDeletedPosition() != null && !c1.getMarkDeletedPosition().equals(previousPos), - 5, 100); - previousMarkDelete = c1.getMarkDeletedPosition(); - } - - PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); - assertEquals(lastLedgerInfo.getLedgerId(), markDeletePosition.getLedgerId()); - assertEquals(lastLedgerInfo.getEntries() - 1, markDeletePosition.getEntryId()); + assertTrue(monitor.expireMessages(ttlSeconds)); + Awaitility.await().untilAsserted(() -> { + PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); + // The markDeletePosition points to the last entry of the previous ledger in lastLedgerInfo. + assertEquals(markDeletePosition.getLedgerId(), lastLedgerInfo.getLedgerId() - 1); + assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); + }); c1.close(); ledger.close(); @@ -440,20 +437,25 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test(groups = "flaky") + @Test public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; + int entriesNum = 10; ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); // set client clock to 10 days later long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entriesNum; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); } - assertEquals(ledger.getLedgersInfoAsList().size(), 10); + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); + // The number of ledgers should be (entriesNum / MaxEntriesPerLedger) + 1 + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(ledger.getLedgersInfoAsList().size(), entriesNum + 1); PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); From 9d72e6bd847df85a7d18f1827274df96a446798f Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 15 Apr 2024 16:15:59 +0800 Subject: [PATCH 24/55] [improve][test] Add topic operation checker for topic API (#22468) --- .../pulsar/broker/admin/TopicAuthZTest.java | 156 +++++++++++++++--- 1 file changed, 135 insertions(+), 21 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index d09bc0a3ffde1..e6ff0ce2bb43a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,9 +20,18 @@ package org.apache.pulsar.broker.admin; import io.jsonwebtoken.Jwts; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -37,22 +46,21 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; @Test(groups = "broker-admin") public class TopicAuthZTest extends MockedPulsarStandalone { @@ -61,13 +69,17 @@ public class TopicAuthZTest extends MockedPulsarStandalone { private PulsarAdmin tenantManagerAdmin; + private AuthorizationService authorizationService; + + private AuthorizationService orignalAuthorizationService; + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows @BeforeClass(alwaysRun = true) - public void before() { + public void setup() { configureTokenAuthentication(); configureDefaultAuthorization(); enableTransaction(); @@ -99,7 +111,7 @@ protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws @SneakyThrows @AfterClass(alwaysRun = true) - public void after() { + public void cleanup() { if (superUserAdmin != null) { superUserAdmin.close(); } @@ -109,6 +121,51 @@ public void after() { close(); } + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + private AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof TopicOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else if (operation instanceof NamespaceOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + + return execFlag; + } + @DataProvider(name = "partitioned") public static Object[][] partitioned() { return new Object[][] { @@ -204,6 +261,8 @@ public void testGetManagedLedgerInfo() { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getInternalInfo(topic)); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); + for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (action == AuthAction.produce || action == AuthAction.consume) { @@ -214,6 +273,9 @@ public void testGetManagedLedgerInfo() { } superUserAdmin.topics().revokePermissions(topic, subject); } + + Assert.assertTrue(execFlag.get()); + superUserAdmin.topics().deletePartitionedTopic(topic, true); } @@ -244,8 +306,10 @@ public void testGetPartitionedStatsAndInternalStats() { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedStats(topic, false)); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedInternalStats(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -313,14 +377,20 @@ public void testCreateSubscriptionAndUpdateSubscriptionPropertiesAndAnalyzeSubsc tenantManagerAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty()); // test nobody + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getSubscriptionProperties(topic, "test-sub")); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty())); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -362,10 +432,15 @@ public void testCreateMissingPartition() { superUserAdmin.topics().createMissedPartitions(topic); // test tenant manager + + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.CREATE_TOPIC); tenantManagerAdmin.topics().createMissedPartitions(topic); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().createMissedPartitions(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -396,8 +471,10 @@ public void testPartitionedTopicMetadata(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getPartitionedTopicMetadata(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -434,16 +511,18 @@ public void testGetProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getProperties(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getProperties(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.produce == action || AuthAction.consume == action) { - subAdmin.topics().getPartitionedTopicMetadata(topic); + subAdmin.topics().getProperties(topic); } else { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + () -> subAdmin.topics().getProperties(topic)); } superUserAdmin.topics().revokePermissions(topic, subject); } @@ -472,8 +551,10 @@ public void testUpdateProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().updateProperties(topic, properties); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.UPDATE_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().updateProperties(topic, properties)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -504,8 +585,10 @@ public void testRemoveProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().removeProperties(topic, "key1"); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.DELETE_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().removeProperties(topic, "key1")); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -539,8 +622,11 @@ public void testDeletePartitionedTopic() { tenantManagerAdmin.topics().deletePartitionedTopic(topic); createTopic(topic, true); + + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.DELETE_TOPIC); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().deletePartitionedTopic(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(ns, subject, Set.of(action)); @@ -548,7 +634,6 @@ public void testDeletePartitionedTopic() { () -> subAdmin.topics().deletePartitionedTopic(topic)); superUserAdmin.namespaces().revokePermissionsOnNamespace(ns, subject); } - deleteTopic(topic, true); } @Test(dataProvider = "partitioned") @@ -571,8 +656,10 @@ public void testGetSubscription(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getSubscriptions(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_SUBSCRIPTIONS); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getSubscriptions(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -616,6 +703,7 @@ public void testGetInternalStats(boolean partitioned) { } + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); if (partitioned) { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedInternalStats(topic)); @@ -623,6 +711,7 @@ public void testGetInternalStats(boolean partitioned) { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getInternalStats(topic)); } + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -671,8 +760,10 @@ public void testDeleteSubscription(boolean partitioned) { tenantManagerAdmin.topics().deleteSubscription(topic, subName); superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.UNSUBSCRIBE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().deleteSubscription(topic, subName)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -708,9 +799,10 @@ public void testSkipAllMessage(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().skipAllMessages(topic, subName); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.SKIP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().skipAllMessages(topic, subName)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -746,10 +838,10 @@ public void testSkipMessage() { // test tenant manager tenantManagerAdmin.topics().skipMessages(topic, subName, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.SKIP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().skipMessages(topic, subName, 1)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -782,10 +874,10 @@ public void testExpireMessagesForAllSubscriptions(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().expireMessagesForAllSubscriptions(topic, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessagesForAllSubscriptions(topic, 1)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -820,10 +912,10 @@ public void testResetCursor(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis()); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.RESET_CURSOR); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis())); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -858,10 +950,10 @@ public void testResetCursorOnPosition() { // test tenant manager tenantManagerAdmin.topics().resetCursor(topic, subName, MessageId.latest); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.RESET_CURSOR); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().resetCursor(topic, subName, MessageId.latest)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -903,8 +995,10 @@ public void testGetMessageById() { // test tenant manager tenantManagerAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId()); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId())); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -947,9 +1041,10 @@ public void testPeekNthMessage() { // test tenant manager tenantManagerAdmin.topics().peekMessages(topic, subName, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().peekMessages(topic, subName, 1)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -992,8 +1087,10 @@ public void testExamineMessage() { // test tenant manager tenantManagerAdmin.topics().examineMessage(topic, "latest", 1); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().examineMessage(topic, "latest", 1)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1039,7 +1136,9 @@ public void testExpireMessage(boolean partitioned) { superUserAdmin.topics().expireMessages(topic, subName, 1); // test tenant manager + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); tenantManagerAdmin.topics().expireMessages(topic, subName, 1); + Assert.assertTrue(execFlag.get()); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessages(topic, subName, 1)); @@ -1090,8 +1189,10 @@ public void testExpireMessageByPosition() { // test tenant manager tenantManagerAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1294,6 +1395,15 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer.accept(subAdmin)); } + AtomicBoolean execFlag = null; + if (topicOpType == OperationAuthType.Lookup) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); + } else if (topicOpType == OperationAuthType.Produce) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); + } else if (topicOpType == OperationAuthType.Consume) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); + } + for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); @@ -1305,6 +1415,10 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer Date: Tue, 16 Apr 2024 00:21:04 +0900 Subject: [PATCH 25/55] [fix][sec] Upgrade Bouncycastle to 1.78 (#22509) Co-authored-by: hoguni --- bouncy-castle/bc/LICENSE | 6 +++--- distribution/server/src/assemble/LICENSE.bin.txt | 8 ++++---- distribution/shell/src/assemble/LICENSE.bin.txt | 8 ++++---- pom.xml | 2 +- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index dae8f16df5b82..14f4e76e921d3 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,6 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index a409ad07ed1b4..4dc6e4341672c 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -613,10 +613,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcutil-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcutil-jdk18on-1.78.jar ------------------------ diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 3ac489fa49a68..069e61b89b55a 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -473,10 +473,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar ------------------------ diff --git a/pom.xml b/pom.xml index 47ac21b62bfed..8a43e536cdb03 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ flexible messaging model and an intuitive client API. 1.7.32 4.4 2.23.1 - 1.75 + 1.78 1.0.6 1.0.2.4 2.14.2 From bbff29d8ecc2f6c7ec91e0a48085fe14c8ffd6b8 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 16 Apr 2024 08:04:11 +0800 Subject: [PATCH 26/55] [fix][io] Kafka Source connector maybe stuck (#22511) --- .../pulsar/io/kafka/KafkaAbstractSource.java | 28 +++++- .../kafka/source/KafkaAbstractSourceTest.java | 89 +++++++++++++++++++ 2 files changed, 116 insertions(+), 1 deletion(-) diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index 782f9d5d57dbb..7eba7438b2b1d 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -63,6 +64,7 @@ public abstract class KafkaAbstractSource extends PushSource { private volatile boolean running = false; private KafkaSourceConfig kafkaSourceConfig; private Thread runnerThread; + private long maxPollIntervalMs; @Override public void open(Map config, SourceContext sourceContext) throws Exception { @@ -126,6 +128,13 @@ public void open(Map config, SourceContext sourceContext) throws props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceConfig.getAutoOffsetReset()); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getKeyDeserializationClass()); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getValueDeserializationClass()); + if (props.containsKey(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG)) { + maxPollIntervalMs = Long.parseLong(props.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG).toString()); + } else { + maxPollIntervalMs = Long.parseLong( + ConsumerConfig.configDef().defaultValues().get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG) + .toString()); + } try { consumer = new KafkaConsumer<>(beforeCreateConsumer(props)); } catch (Exception ex) { @@ -175,7 +184,9 @@ public void start() { index++; } if (!kafkaSourceConfig.isAutoCommitEnabled()) { - CompletableFuture.allOf(futures).get(); + // Wait about 2/3 of the time of maxPollIntervalMs. + // so as to avoid waiting for the timeout to be kicked out of the consumer group. + CompletableFuture.allOf(futures).get(maxPollIntervalMs * 2 / 3, TimeUnit.MILLISECONDS); consumer.commitSync(); } } catch (Exception e) { @@ -253,6 +264,21 @@ public void ack() { completableFuture.complete(null); } + @Override + public void fail() { + completableFuture.completeExceptionally( + new RuntimeException( + String.format( + "Failed to process record with kafka topic: %s partition: %d offset: %d key: %s", + record.topic(), + record.partition(), + record.offset(), + getKey() + ) + ) + ); + } + @Override public Schema getSchema() { return schema; diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 7675de0636e8a..6b4719709a178 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -21,12 +21,18 @@ import com.google.common.collect.ImmutableMap; import java.time.Duration; import java.util.Collections; +import java.util.Arrays; import java.lang.reflect.Field; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.kafka.KafkaAbstractSource; import org.apache.pulsar.io.kafka.KafkaSourceConfig; @@ -46,6 +52,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import static org.testng.Assert.fail; @@ -218,6 +225,88 @@ public final void throwExceptionByPoll() throws Exception { source.read(); } + @Test + public final void throwExceptionBySendFail() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 300000); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail + Record record = source.read(); + record.fail(); + + // read again will throw RuntimeException. + try { + source.read(); + fail("Should throw exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertTrue(e.getCause().getMessage().contains("Failed to process record with kafka topic")); + } + } + + @Test + public final void throwExceptionBySendTimeOut() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 1); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail, just read do noting. + source.read(); + + // read again will throw TimeOutException. + try { + source.read(); + fail("Should throw exception"); + } catch (Exception e) { + assertTrue(e instanceof TimeoutException); + } + } + private File getFile(String name) { ClassLoader classLoader = getClass().getClassLoader(); return new File(classLoader.getResource(name).getFile()); From 203f305bf449dd335b39501177f210cfcb73d5fa Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 16 Apr 2024 00:34:59 -0700 Subject: [PATCH 27/55] [fix][broker] Fix Replicated Topic unload bug when ExtensibleLoadManager is enabled (#22496) --- .../channel/ServiceUnitStateChannelImpl.java | 13 ++++++------- .../broker/namespace/NamespaceService.java | 5 +++++ .../nonpersistent/NonPersistentTopic.java | 3 ++- .../service/persistent/PersistentTopic.java | 3 ++- .../service/ReplicatorGlobalNSTest.java | 16 ++++++++++++++++ .../broker/service/ReplicatorTestBase.java | 19 +++++++++++++++++-- 6 files changed, 48 insertions(+), 11 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 68b38080e73a1..e355187af4ba2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -88,7 +88,6 @@ import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -1381,8 +1380,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight non-system bundle override messages.", e); } @@ -1405,8 +1404,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight system bundle override messages.", e); } @@ -1584,8 +1583,8 @@ protected void monitorOwnerships(List brokers) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight messages.", e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 7c62f264c78d4..3e7bb9560e327 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -824,6 +824,11 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, } public CompletableFuture isNamespaceBundleOwned(NamespaceBundle bundle) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); + return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) + .thenApply(Optional::isPresent); + } return pulsar.getLocalMetadataStore().exists(ServiceUnitUtils.path(bundle)); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 0ac06d6883ff1..9a3a0a7d83d50 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -589,7 +589,8 @@ public CompletableFuture stopReplProducers() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e4441969101c1..936091edce557 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1704,7 +1704,8 @@ CompletableFuture checkPersistencePolicies() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index 4296f3f416868..eed849ef1a01e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.Sets; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -32,6 +34,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; import java.lang.reflect.Method; @@ -41,6 +45,18 @@ public class ReplicatorGlobalNSTest extends ReplicatorTestBase { protected String methodName; + @DataProvider(name = "loadManagerClassName") + public static Object[][] loadManagerClassName() { + return new Object[][]{ + {ModularLoadManagerImpl.class.getName()}, + {ExtensibleLoadManagerImpl.class.getName()} + }; + } + + @Factory(dataProvider = "loadManagerClassName") + public ReplicatorGlobalNSTest(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; + } @BeforeMethod public void beforeMethod(Method m) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index 11d663ff9f4f4..ba9f850ff0cc1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -119,6 +119,11 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { protected final String cluster2 = "r2"; protected final String cluster3 = "r3"; protected final String cluster4 = "r4"; + protected String loadManagerClassName; + + protected String getLoadManagerClassName() { + return loadManagerClassName; + } // Default frequency public int getBrokerServicePurgeInactiveFrequency() { @@ -271,8 +276,9 @@ protected void setup() throws Exception { .brokerClientTlsTrustStoreType(keyStoreType) .build()); - admin1.tenants().createTenant("pulsar", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r2", "r3"))); + updateTenantInfo("pulsar", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), + Sets.newHashSet("r1", "r2", "r3"))); admin1.namespaces().createNamespace("pulsar/ns", Sets.newHashSet("r1", "r2", "r3")); admin1.namespaces().createNamespace("pulsar/ns1", Sets.newHashSet("r1", "r2")); @@ -344,6 +350,7 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadManagerClassName(getLoadManagerClassName()); } public void resetConfig1() { @@ -409,6 +416,14 @@ protected void cleanup() throws Exception { resetConfig4(); } + protected void updateTenantInfo(String tenant, TenantInfoImpl tenantInfo) throws Exception { + if (!admin1.tenants().getTenants().contains(tenant)) { + admin1.tenants().createTenant(tenant, tenantInfo); + } else { + admin1.tenants().updateTenant(tenant, tenantInfo); + } + } + static class MessageProducer implements AutoCloseable { URL url; String namespace; From 70b401b1de9df685283140cff1f83252abc27045 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Tue, 16 Apr 2024 19:53:29 +0800 Subject: [PATCH 28/55] [improve][fn] Expose `RuntimeFlags` as CLI option for Pulsar Functions and Connectors (#22514) --- .../java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java | 4 +++- .../java/org/apache/pulsar/admin/cli/CmdFunctions.java | 7 +++++++ .../main/java/org/apache/pulsar/admin/cli/CmdSinks.java | 6 ++++++ .../main/java/org/apache/pulsar/admin/cli/CmdSources.java | 6 ++++++ 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java index 4d906af9424f5..d3087b7fc873c 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java @@ -165,7 +165,8 @@ public void testCreateFunction() throws Exception { "--className", DummyFunction.class.getName(), "--dead-letter-topic", "test-dead-letter-topic", "--custom-runtime-options", "custom-runtime-options", - "--user-config", "{\"key\": [\"value1\", \"value2\"]}" + "--user-config", "{\"key\": [\"value1\", \"value2\"]}", + "--runtime-flags", "--add-opens java.base/java.lang=ALL-UNNAMED" }); CreateFunction creater = cmd.getCreater(); @@ -175,6 +176,7 @@ public void testCreateFunction() throws Exception { assertEquals(Boolean.FALSE, creater.getAutoAck()); assertEquals("test-dead-letter-topic", creater.getDeadLetterTopic()); assertEquals("custom-runtime-options", creater.getCustomRuntimeOptions()); + assertEquals("--add-opens java.base/java.lang=ALL-UNNAMED", creater.getRuntimeFlags()); verify(functions, times(1)).createFunction(any(FunctionConfig.class), anyString()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java index 15b8fca076104..5e80c168d920b 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java @@ -377,6 +377,9 @@ abstract class FunctionDetailsCommand extends BaseCommand { @Option(names = "--dead-letter-topic", description = "The topic where messages that are not processed successfully are sent to #Java") protected String deadLetterTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected FunctionConfig functionConfig; protected String userCodeFile; @@ -676,6 +679,10 @@ void processArguments() throws Exception { userCodeFile = functionConfig.getGo(); } + if (null != runtimeFlags) { + functionConfig.setRuntimeFlags(runtimeFlags); + } + // check if configs are valid validateFunctionConfigs(functionConfig); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java index f3172a49b0154..be1cd0af96085 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java @@ -404,6 +404,9 @@ abstract class SinkDetailsCommand extends BaseCommand { protected String transformFunctionConfig; @Option(names = "--log-topic", description = "The topic to which the logs of a Pulsar Sink are produced") protected String logTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected SinkConfig sinkConfig; @@ -602,6 +605,9 @@ void processArguments() throws Exception { if (null != logTopic) { sinkConfig.setLogTopic(logTopic); } + if (null != runtimeFlags) { + sinkConfig.setRuntimeFlags(runtimeFlags); + } // check if configs are valid validateSinkConfigs(sinkConfig); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java index 03df3903a6c16..e691d7c126778 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java @@ -359,6 +359,9 @@ abstract class SourceDetailsCommand extends BaseCommand { protected String secretsString; @Option(names = "--log-topic", description = "The topic to which the logs of a Pulsar Sink are produced") protected String logTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected SourceConfig sourceConfig; @@ -497,6 +500,9 @@ void processArguments() throws Exception { if (null != logTopic) { sourceConfig.setLogTopic(logTopic); } + if (null != runtimeFlags) { + sourceConfig.setRuntimeFlags(runtimeFlags); + } // check if source configs are valid validateSourceConfigs(sourceConfig); From ffdfc0c4e0881c682132e79c3cbf9768b1ab4f89 Mon Sep 17 00:00:00 2001 From: sinan liu Date: Tue, 16 Apr 2024 21:19:44 +0800 Subject: [PATCH 29/55] [fix][test] SchemaMap in AutoConsumeSchema has been reused (#22500) --- .../api/SimpleProducerConsumerTest.java | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 7552b84a1c553..691f501777eda 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4329,6 +4329,10 @@ public static Object[] avroSchemaProvider() { public void testAccessAvroSchemaMetadata(Schema schema) throws Exception { log.info("-- Starting {} test --", methodName); + if (pulsarClient == null) { + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } + final String topic = "persistent://my-property/my-ns/accessSchema"; Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) .topic(topic) @@ -4344,37 +4348,43 @@ public void testAccessAvroSchemaMetadata(Schema schema) throws Exception producer.send(payload); producer.close(); - GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); - consumer.close(); - assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); - org.apache.avro.generic.GenericRecord nativeAvroRecord = null; - JsonNode nativeJsonRecord = null; - if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { - nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); - assertNotNull(nativeAvroRecord); - } else { - nativeJsonRecord = (JsonNode) res.getNativeObject(); - assertNotNull(nativeJsonRecord); - } - for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { - log.info("field {} {}", f.getName(), res.getField(f)); - assertEquals("field", f.getName()); - assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); - - if (nativeAvroRecord != null) { - // test that the native schema is accessible - org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); - // a nullable string is an UNION - assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + try { + GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); + consumer.close(); + assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); + org.apache.avro.generic.GenericRecord nativeAvroRecord = null; + JsonNode nativeJsonRecord = null; + if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { + nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); + assertNotNull(nativeAvroRecord); } else { - assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + nativeJsonRecord = (JsonNode) res.getNativeObject(); + assertNotNull(nativeJsonRecord); + } + for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { + log.info("field {} {}", f.getName(), res.getField(f)); + assertEquals("field", f.getName()); + assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); + + if (nativeAvroRecord != null) { + // test that the native schema is accessible + org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); + // a nullable string is an UNION + assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + } else { + assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + } } + assertEquals(1, res.getFields().size()); + } catch (Exception e) { + fail(); + } finally { + pulsarClient.shutdown(); + pulsarClient = null; + admin.schemas().deleteSchema(topic); } - assertEquals(1, res.getFields().size()); - - admin.schemas().deleteSchema(topic); } @Test(timeOut = 100000) From 4ca4e2855267e3b36ee1a27f7144b89ba9194821 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:07:30 +0800 Subject: [PATCH 30/55] [improve] Make the config `metricsBufferResponse` description more effective (#22490) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 38a4c552f0b6b..2b58cbc2d1178 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 @@ -2928,8 +2928,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece private boolean exposeTopicLevelMetricsInPrometheus = true; @FieldContext( category = CATEGORY_METRICS, - doc = "If true, export buffered metrics" - ) + doc = "Set to true to enable the broker to cache the metrics response; the default is false. " + + "The caching period is defined by `managedLedgerStatsPeriodSeconds`. " + + "The broker returns the same response for subsequent requests within the same period. " + + "Ensure that the scrape interval of your monitoring system matches the caching period.") private boolean metricsBufferResponse = false; @FieldContext( category = CATEGORY_METRICS, From d5b36da9a2e0d4f17bea8e033180e494e93dc442 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:12:34 +0800 Subject: [PATCH 31/55] [improve][broker] Add topic name to emitted error messages. (#22506) --- .../pulsar/broker/service/AbstractTopic.java | 17 +++++++++-------- .../pulsar/broker/admin/AdminApi2Test.java | 6 ++++-- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 05defa60c050b..e772486fcc6ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -867,7 +867,7 @@ protected CompletableFuture> incrementTopicEpochIfNeeded(Producer } } catch (Exception e) { - log.error("Encountered unexpected error during exclusive producer creation", e); + log.error("[{}] Encountered unexpected error during exclusive producer creation", topic, e); return FutureUtil.failedFuture(new BrokerServiceException(e)); } finally { lock.writeLock().unlock(); @@ -941,14 +941,14 @@ protected void checkTopicFenced() throws BrokerServiceException { protected CompletableFuture internalAddProducer(Producer producer) { if (isProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max producers limit")); } if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max same address producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max same address producers limit")); } if (log.isDebugEnabled()) { @@ -983,7 +983,7 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr if (previousIsActive.isEmpty() || previousIsActive.get()) { return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( "Producer with name '" + newProducer.getProducerName() - + "' is already connected to topic")); + + "' is already connected to topic '" + topic + "'")); } else { // If the connection of the previous producer is not active, the method // "cnx().checkConnectionLiveness()" will trigger the close for it and kick off the previous @@ -996,7 +996,8 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr }); } return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( - "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic")); + "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic '" + + topic + "'")); } } @@ -1346,7 +1347,7 @@ public static Optional getMigratedClusterUrl(PulsarService pulsar, S return getMigratedClusterUrlAsync(pulsar, topic) .get(pulsar.getPulsarResources().getClusterResources().getOperationTimeoutSec(), TimeUnit.SECONDS); } catch (Exception e) { - log.warn("Failed to get migration cluster URL", e); + log.warn("[{}] Failed to get migration cluster URL", topic, e); } return Optional.empty(); } 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 6bc64f4dd65d0..249dd3c4607be 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 @@ -2888,7 +2888,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //set the limit to 3 admin.namespaces().setMaxProducersPerTopic(myNamespace, 3); @@ -2902,7 +2903,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer1 = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //clean up From 1dd82a0affd6ec3686fa85d444c35bbbb4e9ce12 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 17 Apr 2024 18:14:38 +0800 Subject: [PATCH 32/55] [improve][broker] Repeat the handleMetadataChanges callback when configurationMetadataStore equals localMetadataStore (#22519) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 2687532693a45..249008bad91ad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -357,7 +357,9 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.entryFilterProvider = new EntryFilterProvider(pulsar.getConfiguration()); pulsar.getLocalMetadataStore().registerListener(this::handleMetadataChanges); - pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + if (pulsar.getConfigurationMetadataStore() != pulsar.getLocalMetadataStore()) { + pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + } this.inactivityMonitor = OrderedScheduler.newSchedulerBuilder() .name("pulsar-inactivity-monitor") From 94f6c7ccd2bf8bc261d45ab41f6c7f123359fa47 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 03:15:01 -0700 Subject: [PATCH 33/55] [improve][broker] Optimize gzip compression for /metrics endpoint by sharing/caching compressed result (#22521) --- .../prometheus/PrometheusMetricsServlet.java | 1 + .../pulsar/broker/web/GzipHandlerUtil.java | 21 +++ .../broker/web/GzipHandlerUtilTest.java | 36 ++++ .../apache/pulsar/broker/PulsarService.java | 3 +- .../PrometheusMetricsGenerator.java | 176 ++++++++++++++++-- .../PulsarPrometheusMetricsServlet.java | 28 ++- .../pulsar/PrometheusMetricsTestUtil.java | 2 +- 7 files changed, 253 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 8a41bed29d44f..8685348174cd6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { + public static final String DEFAULT_METRICS_PATH = "/metrics"; private static final long serialVersionUID = 1L; static final int HTTP_STATUS_OK_200 = 200; static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java index 37c9c05e5d53c..9e980cecb791f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -19,8 +19,10 @@ package org.apache.pulsar.broker.web; import java.util.List; +import org.eclipse.jetty.http.pathmap.PathSpecSet; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.util.IncludeExclude; public class GzipHandlerUtil { public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { @@ -45,4 +47,23 @@ public static boolean isGzipCompressionCompletelyDisabled(List gzipCompr && (gzipCompressionExcludedPaths.get(0).equals("^.*") || gzipCompressionExcludedPaths.get(0).equals("^.*$")); } + + /** + * Check if GZIP compression is enabled for the given endpoint. + * @param gzipCompressionExcludedPaths list of paths that should not be compressed + * @param endpoint the endpoint to check + * @return true if GZIP compression is enabled for the endpoint, false otherwise + */ + public static boolean isGzipCompressionEnabledForEndpoint(List gzipCompressionExcludedPaths, + String endpoint) { + if (gzipCompressionExcludedPaths == null || gzipCompressionExcludedPaths.isEmpty()) { + return true; + } + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + return false; + } + IncludeExclude paths = new IncludeExclude<>(PathSpecSet.class); + paths.exclude(gzipCompressionExcludedPaths.toArray(new String[0])); + return paths.test(endpoint); + } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java new file mode 100644 index 0000000000000..d6958695dec9f --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.web; + +import static org.testng.Assert.*; +import java.util.Arrays; +import org.testng.annotations.Test; + +public class GzipHandlerUtilTest { + + @Test + public void testIsGzipCompressionEnabledForEndpoint() { + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(null, "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*$"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics2")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/admin", "/custom"), "/metrics")); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 9f7b40cc38334..7613a13db22de 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 @@ -110,6 +110,7 @@ import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.stats.prometheus.PulsarPrometheusMetricsServlet; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; @@ -1040,7 +1041,7 @@ private void addWebServerHandlers(WebService webService, true, attributeMap, true, Topics.class); // Add metrics servlet - webService.addServlet("/metrics", + webService.addServlet(PrometheusMetricsServlet.DEFAULT_METRICS_PATH, new ServletHolder(metricsServlet), config.isAuthenticateMetricsEndpoint(), attributeMap); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index bbd09335c0a97..8cd68caf1ee26 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -30,6 +30,8 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.CharBuffer; import java.nio.charset.StandardCharsets; import java.time.Clock; @@ -43,6 +45,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.zip.CRC32; +import java.util.zip.Deflater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; @@ -72,7 +76,7 @@ public class PrometheusMetricsGenerator implements AutoCloseable { private volatile boolean closed; public static class MetricsBuffer { - private final CompletableFuture bufferFuture; + private final CompletableFuture bufferFuture; private final long createTimeslot; private final AtomicInteger refCnt = new AtomicInteger(2); @@ -81,7 +85,7 @@ public static class MetricsBuffer { createTimeslot = timeslot; } - public CompletableFuture getBufferFuture() { + public CompletableFuture getBufferFuture() { return bufferFuture; } @@ -113,6 +117,151 @@ public void release() { } } + /** + * A wraps the response buffer and asynchronously provides a gzip compressed buffer when requested. + */ + public static class ResponseBuffer { + private final ByteBuf uncompressedBuffer; + private boolean released = false; + private CompletableFuture compressedBuffer; + + private ResponseBuffer(final ByteBuf uncompressedBuffer) { + this.uncompressedBuffer = uncompressedBuffer; + } + + public ByteBuf getUncompressedBuffer() { + return uncompressedBuffer; + } + + public synchronized CompletableFuture getCompressedBuffer(Executor executor) { + if (released) { + throw new IllegalStateException("Already released!"); + } + if (compressedBuffer == null) { + compressedBuffer = new CompletableFuture<>(); + ByteBuf retainedDuplicate = uncompressedBuffer.retainedDuplicate(); + executor.execute(() -> { + try { + compressedBuffer.complete(compress(retainedDuplicate)); + } catch (Exception e) { + compressedBuffer.completeExceptionally(e); + } finally { + retainedDuplicate.release(); + } + }); + } + return compressedBuffer; + } + + private ByteBuf compress(ByteBuf uncompressedBuffer) { + GzipByteBufferWriter gzipByteBufferWriter = new GzipByteBufferWriter(uncompressedBuffer.alloc(), + uncompressedBuffer.readableBytes()); + return gzipByteBufferWriter.compress(uncompressedBuffer); + } + + public synchronized void release() { + released = true; + uncompressedBuffer.release(); + if (compressedBuffer != null) { + compressedBuffer.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); + } + } + } + + /** + * Compress input nio buffers into gzip format with output in a Netty composite ByteBuf. + */ + private static class GzipByteBufferWriter { + private static final byte[] GZIP_HEADER = + new byte[] {(byte) 0x1f, (byte) 0x8b, Deflater.DEFLATED, 0, 0, 0, 0, 0, 0, 0}; + private final ByteBufAllocator bufAllocator; + private final Deflater deflater; + private final CRC32 crc; + private final int bufferSize; + private final CompositeByteBuf resultBuffer; + private ByteBuf backingCompressBuffer; + private ByteBuffer compressBuffer; + + GzipByteBufferWriter(ByteBufAllocator bufAllocator, int readableBytes) { + deflater = new Deflater(Deflater.DEFAULT_COMPRESSION, true); + crc = new CRC32(); + this.bufferSize = Math.max(Math.min(resolveChunkSize(bufAllocator), readableBytes), 8192); + this.bufAllocator = bufAllocator; + this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 1); + allocateBuffer(); + } + + /** + * Compress the input Netty buffer and append it to the result buffer in gzip format. + * @param uncompressedBuffer + */ + public ByteBuf compress(ByteBuf uncompressedBuffer) { + try { + ByteBuffer[] nioBuffers = uncompressedBuffer.nioBuffers(); + for (int i = 0, nioBuffersLength = nioBuffers.length; i < nioBuffersLength; i++) { + ByteBuffer nioBuffer = nioBuffers[i]; + compressAndAppend(nioBuffer, i == 0, i == nioBuffersLength - 1); + } + return resultBuffer; + } finally { + close(); + } + } + + private void compressAndAppend(ByteBuffer nioBuffer, boolean isFirst, boolean isLast) { + if (isFirst) { + // write gzip header + compressBuffer.put(GZIP_HEADER); + } + nioBuffer.mark(); + crc.update(nioBuffer); + nioBuffer.reset(); + deflater.setInput(nioBuffer); + if (isLast) { + deflater.finish(); + } + while (!deflater.needsInput() && !deflater.finished()) { + int written = deflater.deflate(compressBuffer); + if (written == 0 && !compressBuffer.hasRemaining()) { + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + allocateBuffer(); + } + } + if (isLast) { + // write gzip footer, integer values are in little endian byte order + compressBuffer.order(ByteOrder.LITTLE_ENDIAN); + // write CRC32 checksum + compressBuffer.putInt((int) crc.getValue()); + // write uncompressed size + compressBuffer.putInt(deflater.getTotalIn()); + // append the last compressed buffer + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + backingCompressBuffer = null; + compressBuffer = null; + } + } + + private void allocateBuffer() { + backingCompressBuffer = bufAllocator.directBuffer(bufferSize); + compressBuffer = backingCompressBuffer.nioBuffer(0, bufferSize); + } + + private void close() { + if (deflater != null) { + deflater.end(); + } + if (backingCompressBuffer != null) { + backingCompressBuffer.release(); + } + } + } + private final PulsarService pulsar; private final boolean includeTopicMetrics; private final boolean includeConsumerMetrics; @@ -187,13 +336,7 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used // for allocating the buffers ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; - int chunkSize; - if (byteBufAllocator instanceof PooledByteBufAllocator) { - PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; - chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); - } else { - chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; - } + int chunkSize = resolveChunkSize(byteBufAllocator); CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); int totalLen = 0; @@ -204,6 +347,17 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { return buf; } + private static int resolveChunkSize(ByteBufAllocator byteBufAllocator) { + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + return chunkSize; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -335,10 +489,10 @@ public MetricsBuffer renderToBuffer(Executor executor, List bufferFuture = newMetricsBuffer.getBufferFuture(); + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); executor.execute(() -> { try { - bufferFuture.complete(generate0(metricsProviders)); + bufferFuture.complete(new ResponseBuffer(generate0(metricsProviders))); } catch (Exception e) { bufferFuture.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 7fcc74e965c24..43514d481dcab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.web.GzipHandlerUtil.isGzipCompressionEnabledForEndpoint; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.time.Clock; +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; @@ -40,6 +43,7 @@ public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { private static final int EXECUTOR_MAX_THREADS = 4; private final PrometheusMetricsGenerator prometheusMetricsGenerator; + private final boolean gzipCompressionEnabledForMetrics; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, boolean includeProducerMetrics, @@ -50,6 +54,8 @@ public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopic prometheusMetricsGenerator = new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); + gzipCompressionEnabledForMetrics = isGzipCompressionEnabledForEndpoint( + pulsar.getConfiguration().getHttpServerGzipCompressionExcludedPaths(), DEFAULT_METRICS_PATH); } @@ -100,7 +106,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { context.complete(); return; } - metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + boolean compressOutput = gzipCompressionEnabledForMetrics && isGzipAccepted(request); + metricsBuffer.getBufferFuture().thenCompose(responseBuffer -> { + if (compressOutput) { + return responseBuffer.getCompressedBuffer(executor); + } else { + return CompletableFuture.completedFuture(responseBuffer.getUncompressedBuffer()); + } + }).whenComplete((buffer, ex) -> executor.execute(() -> { try { long elapsedNanos = System.nanoTime() - startNanos; // check if the request has been timed out, implement a soft timeout @@ -133,6 +146,9 @@ public void onStartAsync(AsyncEvent event) throws IOException { } else { response.setStatus(HTTP_STATUS_OK_200); response.setContentType("text/plain;charset=utf-8"); + if (compressOutput) { + response.setHeader("Content-Encoding", "gzip"); + } ServletOutputStream outputStream = response.getOutputStream(); if (outputStream instanceof HttpOutput) { HttpOutput output = (HttpOutput) outputStream; @@ -156,4 +172,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { } })); } + + private boolean isGzipAccepted(HttpServletRequest request) { + String acceptEncoding = request.getHeader("Accept-Encoding"); + if (acceptEncoding != null) { + return Arrays.stream(acceptEncoding.split(",")) + .map(String::trim) + .anyMatch(str -> "gzip".equalsIgnoreCase(str)); + } + return false; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java index fcc3b6aa88fb4..68826372b7bd6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -55,7 +55,7 @@ public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputS try { ByteBuf buffer = null; try { - buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS).getUncompressedBuffer(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); From 8ca01cd42edfd4efd986f752f6f8538ea5bf4f94 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 17 Apr 2024 18:46:22 +0800 Subject: [PATCH 34/55] [improve][admin] Align the auth and check it at the first place for topic related API (#22507) --- .../admin/impl/PersistentTopicsBase.java | 419 ++++++++---------- .../broker/admin/v2/PersistentTopics.java | 44 +- .../pulsar/broker/admin/TopicAuthZTest.java | 257 +++++++++-- 3 files changed, 447 insertions(+), 273 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 ab74b1e2bcc0e..1f8d06571908e 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 @@ -128,8 +128,6 @@ import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PolicyName; -import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -2727,14 +2725,14 @@ public String toString() { } protected CompletableFuture internalGetMessageIdByTimestampAsync(long timestamp, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES); return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { if (topicName.isPartitioned()) { return CompletableFuture.completedFuture(null); } else { @@ -2748,7 +2746,6 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long }); } }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { if (!(topic instanceof PersistentTopic)) { @@ -3158,65 +3155,56 @@ protected CompletableFuture> in protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, MessageIdImpl messageId, boolean authoritative) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - log.warn("[{}] Not supported calculate backlog size operation on partitioned-topic {}", - clientAppId(), topicName); - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, - "calculate backlog size is not allowed for partitioned-topic")); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - CompletableFuture future; - if (topicName.isGlobal()) { - future = ret.thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(namespaceName)); - } else { - future = ret; - } - future.thenAccept(__ -> validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(unused -> validateTopicOperationAsync(topicName, - TopicOperation.GET_BACKLOG_SIZE)) - .thenCompose(unused -> getTopicReferenceAsync(topicName)) - .thenAccept(t -> { - PersistentTopic topic = (PersistentTopic) t; - PositionImpl pos = new PositionImpl(messageId.getLedgerId(), - messageId.getEntryId()); - if (topic == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - return; - } - ManagedLedgerImpl managedLedger = - (ManagedLedgerImpl) topic.getManagedLedger(); - if (messageId.getLedgerId() == -1) { - asyncResponse.resume(managedLedger.getTotalSize()); - } else { - asyncResponse.resume(managedLedger.getEstimatedBacklogSize(pos)); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), - topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - })).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to validate global namespace ownership " - + "to get backlog size for topic {}", clientAppId(), topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.GET_BACKLOG_SIZE); + ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + log.warn("[{}] Not supported calculate backlog size operation on partitioned-topic {}", + clientAppId(), topicName); + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, + "calculate backlog size is not allowed for partitioned-topic")); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(unused -> getTopicReferenceAsync(topicName)) + .thenAccept(t -> { + PersistentTopic topic = (PersistentTopic) t; + PositionImpl pos = new PositionImpl(messageId.getLedgerId(), + messageId.getEntryId()); + if (topic == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + return; + } + ManagedLedgerImpl managedLedger = + (ManagedLedgerImpl) topic.getManagedLedger(); + if (messageId.getLedgerId() == -1) { + asyncResponse.resume(managedLedger.getTotalSize()); + } else { + asyncResponse.resume(managedLedger.getEstimatedBacklogSize(pos)); + } + }).exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to validate global namespace ownership " + + "to get backlog size for topic {}", clientAppId(), topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQuotaType backlogQuotaType, @@ -3224,8 +3212,7 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu BacklogQuota.BacklogQuotaType finalBacklogQuotaType = backlogQuotaType == null ? BacklogQuota.BacklogQuotaType.destination_storage : backlogQuotaType; - return validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal)) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); @@ -3266,9 +3253,7 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu } protected CompletableFuture internalSetReplicationClusters(List clusterIds) { - - return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> { if (CollectionUtils.isEmpty(clusterIds)) { throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); @@ -3306,22 +3291,21 @@ protected CompletableFuture internalSetReplicationClusters(List cl } protected CompletableFuture internalRemoveReplicationClusters() { - return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) - .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> { - TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); - topicPolicies.setReplicationClusters(null); - return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies) - .thenRun(() -> { - log.info("[{}] Successfully set replication clusters for namespace={}, " - + "topic={}, clusters={}", - clientAppId(), - namespaceName, - topicName.getLocalName(), - topicPolicies.getReplicationClusters()); - }); - }) - ); + return validatePoliciesReadOnlyAccessAsync() + .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) + .thenCompose(op -> { + TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); + topicPolicies.setReplicationClusters(null); + return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies) + .thenRun(() -> { + log.info("[{}] Successfully set replication clusters for namespace={}, " + + "topic={}, clusters={}", + clientAppId(), + namespaceName, + topicName.getLocalName(), + topicPolicies.getReplicationClusters()); + }); + }); } protected CompletableFuture internalGetDeduplication(boolean applied, boolean isGlobal) { @@ -3683,29 +3667,29 @@ protected CompletableFuture internalTerminateAsync(boolean authoritat "Termination of a system topic is not allowed")); } - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.TERMINATE)) - .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Termination of a partitioned topic is not allowed"); - } - }) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic)) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Termination of a non-persistent topic is not allowed"); - } - return ((PersistentTopic) topic).terminate(); - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.TERMINATE); + return ret.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Termination of a partitioned topic is not allowed"); + } + }) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + if (!(topic instanceof PersistentTopic)) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Termination of a non-persistent topic is not allowed"); + } + return ((PersistentTopic) topic).terminate(); + }); } protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { @@ -3716,73 +3700,63 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); return; } + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.TERMINATE); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(unused -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions == 0) { + String msg = "Termination of a non-partitioned topic is not allowed using partitioned-terminate" + + ", please use terminate commands"; + log.error("[{}] [{}] {}", clientAppId(), topicName, msg); + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); + return; + } + if (partitionMetadata.partitions > 0) { + Map messageIds = new ConcurrentHashMap<>(partitionMetadata.partitions); + final List> futures = + new ArrayList<>(partitionMetadata.partitions); - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.TERMINATE) - .thenCompose(unused -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions == 0) { - String msg = "Termination of a non-partitioned topic is not allowed using partitioned-terminate" - + ", please use terminate commands"; - log.error("[{}] [{}] {}", clientAppId(), topicName, msg); - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); - return; + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + int finalI = i; + futures.add(pulsar().getAdminClient().topics() + .terminateTopicAsync(topicNamePartition.toString()) + .whenComplete((messageId, throwable) -> { + if (throwable != null) { + log.error("[{}] Failed to terminate topic {}", clientAppId(), + topicNamePartition, throwable); + asyncResponse.resume(new RestException(throwable)); + } + messageIds.put(finalI, messageId); + })); + } catch (Exception e) { + log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, + e); + throw new RestException(e); } - if (partitionMetadata.partitions > 0) { - Map messageIds = new ConcurrentHashMap<>(partitionMetadata.partitions); - final List> futures = - new ArrayList<>(partitionMetadata.partitions); - - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - int finalI = i; - futures.add(pulsar().getAdminClient().topics() - .terminateTopicAsync(topicNamePartition.toString()) - .whenComplete((messageId, throwable) -> { - if (throwable != null) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), - topicNamePartition, throwable); - asyncResponse.resume(new RestException(throwable)); - } - messageIds.put(finalI, messageId); - })); - } catch (Exception e) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, - e); - throw new RestException(e); - } + } + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + } else { + log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t); + asyncResponse.resume(new RestException(t)); } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - } else { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t); - asyncResponse.resume(new RestException(t)); - } - } - asyncResponse.resume(messageIds); - return null; - }); } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); + asyncResponse.resume(messageIds); return null; - }) - ).exceptionally(ex -> { + }); + } + }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); @@ -4186,16 +4160,16 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } protected CompletableFuture internalCompactionStatusAsync(boolean authoritative) { - return validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.COMPACT)) + return validateTopicOperationAsync(topicName, TopicOperation.COMPACT) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenApply(topic -> ((PersistentTopic) topic).compactionStatus()); } protected void internalTriggerOffload(AsyncResponse asyncResponse, boolean authoritative, MessageIdImpl messageId) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD)) + validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { try { @@ -4221,8 +4195,8 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD)) + validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { OffloadProcessStatus offloadProcessStatus = ((PersistentTopic) topic).offloadStatus(); @@ -4482,8 +4456,8 @@ private CompletableFuture validateNonPartitionTopicNameAsync(String topicN } protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) + validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { if (topic == null) { @@ -5207,33 +5181,27 @@ private void internalGetReplicatedSubscriptionStatusForNonPartitionedTopic( } protected CompletableFuture internalGetSchemaCompatibilityStrategy(boolean applied) { - CompletableFuture future = validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); if (applied) { - return future.thenCompose(__ -> getSchemaCompatibilityStrategyAsync()); + return getSchemaCompatibilityStrategyAsync(); } - return future - .thenCompose(n -> getTopicPoliciesAsyncWithRetry(topicName).thenApply(op -> { + return getTopicPoliciesAsyncWithRetry(topicName).thenApply(op -> { if (!op.isPresent()) { return null; } SchemaCompatibilityStrategy strategy = op.get().getSchemaCompatibilityStrategy(); return SchemaCompatibilityStrategy.isUndefined(strategy) ? null : strategy; - })); + }); } protected CompletableFuture internalSetSchemaCompatibilityStrategy(SchemaCompatibilityStrategy strategy) { - return validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, - PolicyOperation.WRITE) - .thenCompose((__) -> getTopicPoliciesAsyncWithRetry(topicName) + return getTopicPoliciesAsyncWithRetry(topicName) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); topicPolicies.setSchemaCompatibilityStrategy( strategy == SchemaCompatibilityStrategy.UNDEFINED ? null : strategy); return pulsar().getTopicPoliciesService() .updateTopicPoliciesAsync(topicName, topicPolicies); - })); + }); } protected CompletableFuture internalGetSchemaValidationEnforced(boolean applied) { @@ -5257,54 +5225,47 @@ protected CompletableFuture internalSetSchemaValidationEnforced(boolean sc } protected CompletableFuture internalGetEntryFilters(boolean applied, boolean isGlobal) { - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.READ) - .thenCompose(__ -> { - if (!applied) { - return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) - .thenApply(op -> op.map(TopicPolicies::getEntryFilters).orElse(null)); - } - if (!pulsar().getConfiguration().isAllowOverrideEntryFilters()) { - return CompletableFuture.completedFuture(new EntryFilters(String.join(",", - pulsar().getConfiguration().getEntryFilterNames()))); + if (!applied) { + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + .thenApply(op -> op.map(TopicPolicies::getEntryFilters).orElse(null)); + } + if (!pulsar().getConfiguration().isAllowOverrideEntryFilters()) { + return CompletableFuture.completedFuture(new EntryFilters(String.join(",", + pulsar().getConfiguration().getEntryFilterNames()))); + } + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + .thenApply(op -> op.map(TopicPolicies::getEntryFilters)) + .thenCompose(policyEntryFilters -> { + if (policyEntryFilters.isPresent()) { + return CompletableFuture.completedFuture(policyEntryFilters.get()); } - return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) - .thenApply(op -> op.map(TopicPolicies::getEntryFilters)) - .thenCompose(policyEntryFilters -> { - if (policyEntryFilters.isPresent()) { - return CompletableFuture.completedFuture(policyEntryFilters.get()); + return getNamespacePoliciesAsync(namespaceName) + .thenApply(policies -> policies.entryFilters) + .thenCompose(nsEntryFilters -> { + if (nsEntryFilters != null) { + return CompletableFuture.completedFuture(nsEntryFilters); } - return getNamespacePoliciesAsync(namespaceName) - .thenApply(policies -> policies.entryFilters) - .thenCompose(nsEntryFilters -> { - if (nsEntryFilters != null) { - return CompletableFuture.completedFuture(nsEntryFilters); - } - return CompletableFuture.completedFuture(new EntryFilters(String.join(",", - pulsar().getConfiguration().getEntryFilterNames()))); - }); + return CompletableFuture.completedFuture(new EntryFilters(String.join(",", + pulsar().getConfiguration().getEntryFilterNames()))); }); }); } protected CompletableFuture internalSetEntryFilters(EntryFilters entryFilters, boolean isGlobal) { - - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) - .thenAccept(__ -> validateEntryFilters(entryFilters)) - .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + validateEntryFilters(entryFilters); + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); topicPolicies.setEntryFilters(entryFilters); topicPolicies.setIsGlobal(isGlobal); return pulsar().getTopicPoliciesService() .updateTopicPoliciesAsync(topicName, topicPolicies); - })); + }); } protected CompletableFuture internalRemoveEntryFilters(boolean isGlobal) { - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) - .thenCompose(__ -> - getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) .thenCompose(op -> { if (!op.isPresent()) { return CompletableFuture.completedFuture(null); @@ -5312,7 +5273,7 @@ protected CompletableFuture internalRemoveEntryFilters(boolean isGlobal) { op.get().setEntryFilters(null); op.get().setIsGlobal(isGlobal); return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, op.get()); - })); + }); } protected CompletableFuture validateShadowTopics(List shadowTopics) { @@ -5348,8 +5309,7 @@ protected CompletableFuture internalSetShadowTopic(List shadowTopi return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, "Cannot specify empty shadow topics, please use remove command instead.")); } - return validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> validateShadowTopics(shadowTopics)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) .thenCompose(op -> { @@ -5361,8 +5321,7 @@ protected CompletableFuture internalSetShadowTopic(List shadowTopi } protected CompletableFuture internalDeleteShadowTopics() { - return validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(shadowTopicName -> getTopicPoliciesAsyncWithRetry(topicName)) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 90f0208c81cd6..7e138442ae228 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -2149,7 +2149,8 @@ public void setBacklogQuota( @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, @ApiParam(value = "backlog quota policies for the specified topic") BacklogQuotaImpl backlogQuota) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetBacklogQuota(backlogQuotaType, backlogQuota, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2174,7 +2175,8 @@ public void removeBacklogQuota(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetBacklogQuota(backlogQuotaType, null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2237,7 +2239,8 @@ public void setReplicationClusters( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "List of replication clusters", required = true) List clusterIds) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetReplicationClusters(clusterIds)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2260,7 +2263,8 @@ public void removeReplicationClusters(@Suspended final AsyncResponse asyncRespon @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveReplicationClusters()) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4405,8 +4409,8 @@ public void getSchemaCompatibilityStrategy( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__-> internalGetSchemaCompatibilityStrategy(applied)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -4436,8 +4440,8 @@ public void setSchemaCompatibilityStrategy( @ApiParam(value = "Strategy used to check the compatibility of new schema") SchemaCompatibilityStrategy strategy) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSchemaCompatibilityStrategy(strategy)) .thenRun(() -> { log.info( @@ -4476,8 +4480,8 @@ public void removeSchemaCompatibilityStrategy( @ApiParam(value = "Strategy used to check the compatibility of new schema") SchemaCompatibilityStrategy strategy) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSchemaCompatibilityStrategy(null)) .thenRun(() -> { log.info( @@ -4568,7 +4572,8 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, + "broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetEntryFilters(applied, isGlobal)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -4596,7 +4601,8 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Entry filters for the specified topic") EntryFilters entryFilters) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetEntryFilters(entryFilters, isGlobal)) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4622,7 +4628,8 @@ public void removeEntryFilters(@Suspended final AsyncResponse asyncResponse, + "call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveEntryFilters(isGlobal)) .thenRun(() -> { log.info( @@ -4655,9 +4662,8 @@ public void getShadowTopics( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) - .thenCompose(__ -> validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, - PolicyOperation.READ)) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) .thenAccept(op -> asyncResponse.resume(op.map(TopicPolicies::getShadowTopics).orElse(null))) .exceptionally(ex -> { @@ -4684,7 +4690,8 @@ public void setShadowTopics( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "List of shadow topics", required = true) List shadowTopics) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetShadowTopic(shadowTopics)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4710,7 +4717,8 @@ public void deleteShadowTopics( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalDeleteShadowTopics()) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index e6ff0ce2bb43a..3c0596d531f41 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -19,48 +19,54 @@ package org.apache.pulsar.broker.admin; +import com.google.common.collect.Lists; import io.jsonwebtoken.Jwts; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.commons.lang3.reflect.FieldUtils; -import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.service.plugin.EntryFilterDefinition; +import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.broker.service.plugin.EntryFilterTest; +import org.apache.pulsar.broker.testcontext.MockEntryFilterProvider; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.auth.AuthenticationToken; -import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.security.MockedPulsarStandalone; import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; @Test(groups = "broker-admin") public class TopicAuthZTest extends MockedPulsarStandalone { @@ -1105,15 +1111,15 @@ public void testExamineMessage() { deleteTopic(topic, false); } - @Test(dataProvider = "partitioned", groups = "flaky") + @Test @SneakyThrows - public void testExpireMessage(boolean partitioned) { + public void testExpireMessage() { final String random = UUID.randomUUID().toString(); final String topic = "persistent://public/default/" + random; final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); - createTopic(topic, partitioned); + createTopic(topic, false); @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) @@ -1153,7 +1159,7 @@ public void testExpireMessage(boolean partitioned) { } superUserAdmin.topics().revokePermissions(topic, subject); } - deleteTopic(topic, partitioned); + deleteTopic(topic, false); } @Test @@ -1373,6 +1379,37 @@ public Object[][] authFunction () throws Exception { }; } + @Test + @SneakyThrows + public void testSchemaCompatibility() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + superUserAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, true); + + // test tenant manager + tenantManagerAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, true); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + @Test(dataProvider = "authFunction") public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) throws Exception { @@ -1380,6 +1417,7 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSetEntryFilter() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + final EntryFilterProvider oldEntryFilterProvider = getPulsarService().getBrokerService().getEntryFilterProvider(); + @Cleanup + final MockEntryFilterProvider testEntryFilterProvider = + new MockEntryFilterProvider(getServiceConfiguration()); + + testEntryFilterProvider + .setMockEntryFilters(new EntryFilterDefinition( + "test", + null, + EntryFilterTest.class.getName() + )); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", testEntryFilterProvider, true); + final EntryFilters entryFilter = new EntryFilters("test"); + superUserAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + + // test tenant manager + tenantManagerAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", oldEntryFilterProvider, true); + } + + @Test + @SneakyThrows + public void testRemoveEntryFilter() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + final EntryFilterProvider oldEntryFilterProvider = getPulsarService().getBrokerService().getEntryFilterProvider(); + @Cleanup + final MockEntryFilterProvider testEntryFilterProvider = + new MockEntryFilterProvider(getServiceConfiguration()); + + testEntryFilterProvider + .setMockEntryFilters(new EntryFilterDefinition( + "test", + null, + EntryFilterTest.class.getName() + )); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", testEntryFilterProvider, true); + final EntryFilters entryFilter = new EntryFilters("test"); + superUserAdmin.topicPolicies().removeEntryFiltersPerTopic(topic); + // test tenant manager + tenantManagerAdmin.topicPolicies().removeEntryFiltersPerTopic(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeEntryFiltersPerTopic(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeEntryFiltersPerTopic(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", oldEntryFilterProvider, true); + } + + @Test + @SneakyThrows + public void testShadowTopic() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + String shadowTopic = topic + "-shadow-topic"; + superUserAdmin.topics().createShadowTopic(shadowTopic, topic); + superUserAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic)); + superUserAdmin.topics().getShadowTopics(topic); + superUserAdmin.topics().removeShadowTopics(topic); + + + // test tenant manager + tenantManagerAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic)); + tenantManagerAdmin.topics().getShadowTopics(topic); + tenantManagerAdmin.topics().removeShadowTopics(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getShadowTopics(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getShadowTopics(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + private void createTopic(String topic, boolean partitioned) throws Exception { if (partitioned) { superUserAdmin.topics().createPartitionedTopic(topic, 2); From 56970b714f5adb606b02d12a99db1ceec3fa7832 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 12:46:43 -0700 Subject: [PATCH 35/55] [improve][test] Move ShadowManagedLedgerImplTest to flaky tests (#22526) --- .../bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index cc4b3f2481152..2aa04197ab91e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -51,7 +51,7 @@ private ShadowManagedLedgerImpl openShadowManagedLedger(String name, String sour return (ShadowManagedLedgerImpl) shadowML; } - @Test + @Test(groups = "flaky") public void testShadowWrites() throws Exception { ManagedLedgerImpl sourceML = (ManagedLedgerImpl) factory.open("source_ML", new ManagedLedgerConfig() .setMaxEntriesPerLedger(2) From d0b9d471d53d2db600b55a04d6255688d1fd2d27 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 18 Apr 2024 09:48:14 +0800 Subject: [PATCH 36/55] [fix][broker] Check the broker is available for the SLA monitor bundle when the ExtensibleLoadManager is enabled (#22485) --- .../extensions/ExtensibleLoadManagerImpl.java | 39 ++++++--------- .../broker/namespace/NamespaceService.java | 47 ++++++++++++++----- .../ExtensibleLoadManagerImplTest.java | 43 +++++++++++++++++ 3 files changed, 94 insertions(+), 35 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 0c9448ab69c38..c8cf1c05756a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -492,30 +492,20 @@ public CompletableFuture> assign(Optional { + if (candidateBrokerId != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBrokerId)); + } + return getOrSelectOwnerAsync(serviceUnit, bundle).thenApply(Optional::ofNullable); + }); } return getBrokerLookupData(owner, bundle); }); } - private String getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { - // Check if this is Heartbeat or SLAMonitor namespace - String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); - if (candidateBroker == null) { - candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); - } - if (candidateBroker == null) { - candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); - } - if (candidateBroker != null) { - return candidateBroker.substring(candidateBroker.lastIndexOf('/') + 1); - } - return candidateBroker; + private CompletableFuture getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { + return pulsar.getNamespaceService().getHeartbeatOrSLAMonitorBrokerId(serviceUnit, + cb -> brokerRegistry.lookupAsync(cb).thenApply(Optional::isPresent)); } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, @@ -662,11 +652,12 @@ public CompletableFuture> getOwnershipAsync(Optional { + if (candidateBroker != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBroker)); + } + return serviceUnitStateChannel.getOwnerAsync(bundle); + }); } public CompletableFuture> getOwnershipWithLookupDataAsync(ServiceUnitId bundleUnit) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 3e7bb9560e327..65081f2ea42b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -46,6 +46,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -536,6 +537,38 @@ private CompletableFuture> findBrokerServiceUrl( }); } + /** + * Check if this is Heartbeat or SLAMonitor namespace and return the broker id. + * + * @param serviceUnit the service unit + * @param isBrokerActive the function to check if the broker is active + * @return the broker id + */ + public CompletableFuture getHeartbeatOrSLAMonitorBrokerId( + ServiceUnitId serviceUnit, Function> isBrokerActive) { + String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); + if (candidateBroker != null) { + // Check if the broker is available + final String finalCandidateBroker = candidateBroker; + return isBrokerActive.apply(candidateBroker).thenApply(isActive -> { + if (isActive) { + return finalCandidateBroker; + } else { + return null; + } + }); + } + return CompletableFuture.completedFuture(null); + } + private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture> lookupFuture, LookupOptions options) { @@ -552,17 +585,9 @@ private void searchForCandidateBroker(NamespaceBundle bundle, try { // check if this is Heartbeat or SLAMonitor namespace - candidateBroker = checkHeartbeatNamespace(bundle); - if (candidateBroker == null) { - candidateBroker = checkHeartbeatNamespaceV2(bundle); - } - if (candidateBroker == null) { - String broker = getSLAMonitorBrokerName(bundle); - // checking if the broker is up and running - if (broker != null && isBrokerActive(broker)) { - candidateBroker = broker; - } - } + candidateBroker = getHeartbeatOrSLAMonitorBrokerId(bundle, cb -> + CompletableFuture.completedFuture(isBrokerActive(cb))) + .get(config.getMetadataStoreOperationTimeoutSeconds(), SECONDS); if (candidateBroker == null) { Optional currentLeader = pulsar.getLeaderElectionService().getCurrentLeader(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index e87d6c994cd76..b72ab77e81447 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 @@ -50,6 +50,7 @@ import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -113,6 +114,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -998,6 +1000,47 @@ public void testDeployAndRollbackLoadManager() throws Exception { pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); } } } From 72474d7a2dabdf7acf0b158bd07f1bc8b69b790e Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 17 Apr 2024 23:59:36 -0700 Subject: [PATCH 37/55] [fix][broker] Fix a deadlock in SystemTopicBasedTopicPoliciesService during NamespaceEventsSystemTopicFactory init (#22528) --- .../SystemTopicBasedTopicPoliciesService.java | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 4e9e875bcf4c3..0449e5c885cd3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -34,6 +34,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; +import org.apache.commons.lang3.concurrent.ConcurrentInitializer; +import org.apache.commons.lang3.concurrent.LazyInitializer; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -70,7 +72,19 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final PulsarService pulsarService; private final HashSet localCluster; private final String clusterName; - private volatile NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + private final ConcurrentInitializer + namespaceEventsSystemTopicFactoryLazyInitializer = new LazyInitializer<>() { + @Override + protected NamespaceEventsSystemTopicFactory initialize() { + try { + return new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); + } catch (PulsarServerException e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); + } + } + }; @VisibleForTesting final Map policiesCache = new ConcurrentHashMap<>(); @@ -102,7 +116,7 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { }); }) .buildAsync((namespaceName, executor) -> { - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespaceName); return systemTopicClient.newWriterAsync(); }); @@ -301,7 +315,7 @@ public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicNa result.complete(null); return result; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newReaderAsync().thenAccept(r -> fetchTopicPoliciesAsyncAndCloseReader(r, topicName, null, result)); @@ -373,7 +387,7 @@ protected CompletableFuture> createSystemT } catch (PulsarServerException ex) { return FutureUtil.failedFuture(ex); } - final SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + final SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespace); return systemTopicClient.newReaderAsync(); } @@ -561,7 +575,7 @@ private void refreshTopicPoliciesCache(Message msg) { log.error("Failed to create system topic factory"); break; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newWriterAsync().thenAccept(writer -> writer.deleteAsync(getEventKey(topicName), @@ -595,18 +609,19 @@ private boolean hasReplicateTo(Message message) { } private void createSystemTopicFactoryIfNeeded() throws PulsarServerException { - if (namespaceEventsSystemTopicFactory == null) { - synchronized (this) { - if (namespaceEventsSystemTopicFactory == null) { - try { - namespaceEventsSystemTopicFactory = - new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); - } catch (PulsarServerException e) { - log.error("Create namespace event system topic factory error.", e); - throw e; - } - } - } + try { + getNamespaceEventsSystemTopicFactory(); + } catch (Exception e) { + throw new PulsarServerException(e); + } + } + + private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() { + try { + return namespaceEventsSystemTopicFactoryLazyInitializer.get(); + } catch (Exception e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); } } From 990b8d0334c75255e25899df869887711059cb33 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 18 Apr 2024 07:48:55 -0700 Subject: [PATCH 38/55] [improve][build] Upgrade OWASP Dependency check version to 9.1.0 (#22530) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8a43e536cdb03..c7fba94abd8ea 100644 --- a/pom.xml +++ b/pom.xml @@ -307,7 +307,7 @@ flexible messaging model and an intuitive client API. 0.1.21 1.3 0.4 - 9.0.7 + 9.1.0 0.9.44 1.6.1 6.4.0 From 7acbc4c9f4ce74979a5ae9b6f0721956edaf9295 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 18 Apr 2024 12:35:30 -0700 Subject: [PATCH 39/55] [improve] Run Trivy image scanner with the current image (#22386) --- .../workflows/ci-trivy-container-scan.yaml | 66 ------------------- .github/workflows/pulsar-ci.yaml | 17 +++++ 2 files changed, 17 insertions(+), 66 deletions(-) delete mode 100644 .github/workflows/ci-trivy-container-scan.yaml diff --git a/.github/workflows/ci-trivy-container-scan.yaml b/.github/workflows/ci-trivy-container-scan.yaml deleted file mode 100644 index 47ebe654369d5..0000000000000 --- a/.github/workflows/ci-trivy-container-scan.yaml +++ /dev/null @@ -1,66 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -name: CI - Trivy Container Scan -on: - schedule: - - cron: '0 8 * * *' # Every day at 8am UTC - workflow_dispatch: - inputs: - severity: - description: "Severities to include (comma-separated or 'ALL' to include all)" - required: false - default: 'CRITICAL,HIGH' - -jobs: - container_scan: - if: ${{ github.repository == 'apache/pulsar' }} - name: Trivy Docker image vulnerability scan - runs-on: ubuntu-latest - strategy: - fail-fast: false - matrix: - docker-image: - - 'apachepulsar/pulsar' - docker-tag: - - 'latest' - env: - IMAGE_REF: '${{ matrix.docker-image }}:${{ matrix.docker-tag }}' - steps: - - id: prepare-vars - shell: bash - run: | - IMAGE_REF_CLEAN="$(echo $IMAGE_REF | sed 's/-/_/g; s/\./_/g; s/:/_/g; s/\//_/g')" - echo "image_ref_clean=$IMAGE_REF_CLEAN" >> "$GITHUB_OUTPUT" - echo "report_filename=trivy-scan-$IMAGE_REF_CLEAN.${{ inputs.report-format }}" >> "$GITHUB_OUTPUT" - - name: Run Trivy container scan - uses: aquasecurity/trivy-action@master - with: - image-ref: ${{ env.IMAGE_REF }} - scanners: vuln - severity: ${{ inputs.severity != 'ALL' && inputs.severity || 'UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL' }} - limit-severities-for-sarif: true - format: 'sarif' - output: ${{ steps.prepare-vars.outputs.report_filename }} - exit-code: 1 - - name: Upload Trivy scan results to GitHub Security tab - uses: github/codeql-action/upload-sarif@v2 - if: ${{ failure() }} - with: - sarif_file: '${{ github.workspace }}/${{ steps.prepare-vars.outputs.report_filename }}' diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 22d061ac58094..aa33d7ae197d1 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -876,6 +876,23 @@ jobs: - name: Check binary licenses run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz && src/check-binary-license.sh ./distribution/shell/target/apache-pulsar-shell-*-bin.tar.gz + - name: Run Trivy container scan + uses: aquasecurity/trivy-action@master + if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + with: + image-ref: "apachepulsar/pulsar:latest" + scanners: vuln + severity: CRITICAL,HIGH,MEDIUM,LOW + limit-severities-for-sarif: true + format: 'sarif' + output: 'trivy-results.sarif' + + - name: Upload Trivy scan results to GitHub Security tab + uses: github/codeql-action/upload-sarif@v2 + if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + with: + sarif_file: 'trivy-results.sarif' + - name: Clean up disk space run: | # release disk space since saving docker image consumes local disk space From 7aedb6b20c120ec0a7cc096e33e6305caca26786 Mon Sep 17 00:00:00 2001 From: hanmz Date: Fri, 19 Apr 2024 06:49:18 +0800 Subject: [PATCH 40/55] [fix][broker] Fix typos in Consumer class (#22532) --- .../java/org/apache/pulsar/broker/service/Consumer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 4cd54420200be..6b2028095e205 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -147,7 +147,7 @@ public class Consumer { @Setter private volatile long consumerEpoch; - private long negtiveUnackedMsgsTimestamp; + private long negativeUnackedMsgsTimestamp; @Getter private final SchemaType schemaType; @@ -1102,8 +1102,8 @@ private int addAndGetUnAckedMsgs(Consumer consumer, int ackedMessages) { subscription.addUnAckedMessages(ackedMessages); unackedMsgs = UNACKED_MESSAGES_UPDATER.addAndGet(consumer, ackedMessages); } - if (unackedMsgs < 0 && System.currentTimeMillis() - negtiveUnackedMsgsTimestamp >= 10_000) { - negtiveUnackedMsgsTimestamp = System.currentTimeMillis(); + if (unackedMsgs < 0 && System.currentTimeMillis() - negativeUnackedMsgsTimestamp >= 10_000) { + negativeUnackedMsgsTimestamp = System.currentTimeMillis(); log.warn("unackedMsgs is : {}, ackedMessages : {}, consumer : {}", unackedMsgs, ackedMessages, consumer); } return unackedMsgs; From 2badcf6bd0be1aad2a5ec6da552185b4ef5b745b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 19 Apr 2024 09:13:19 +0800 Subject: [PATCH 41/55] [fix][broker] Fix NPE causing dispatching to stop when using Key_Shared mode and allowOutOfOrderDelivery=true (#22533) --- ...ntStickyKeyDispatcherMultipleConsumers.java | 10 ++++++++++ .../client/api/KeySharedSubscriptionTest.java | 18 ++++++++++++++++-- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index ee2ebd7ca867e..2df9f38531f5d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -457,6 +457,11 @@ private int getAvailablePermits(Consumer c) { @Override protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So skip this filter out. + if (isAllowOutOfOrderDelivery()) { + return src; + } if (src.isEmpty()) { return src; } @@ -501,6 +506,11 @@ protected synchronized NavigableSet filterOutEntriesWillBeDiscarde */ @Override protected boolean hasConsumersNeededNormalRead() { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. + if (isAllowOutOfOrderDelivery()) { + return true; + } for (Consumer consumer : consumerList) { if (consumer == null || consumer.isBlocked()) { continue; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 7219555050839..27aa98597ec12 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -1741,6 +1741,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { admin.topics().delete(topic, false); } + @DataProvider(name = "allowKeySharedOutOfOrder") + public Object[][] allowKeySharedOutOfOrder() { + return new Object[][]{ + {true}, + {false} + }; + } + /** * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. * 1. Start 3 consumers: @@ -1755,8 +1763,8 @@ public void testNoRepeatedReadAndDiscard() throws Exception { * - no repeated Read-and-discard. * - at last, all messages will be received. */ - @Test(timeOut = 180 * 1000) // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. + public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); @@ -1775,6 +1783,8 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { log.info("Published message :{}", messageId); } + KeySharedPolicy keySharedPolicy = KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(allowKeySharedOutOfOrder); // 1. Start 3 consumers and make ack holes. // - one consumer will be closed and trigger a messages redeliver. // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the @@ -1785,18 +1795,21 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); List msgList1 = new ArrayList<>(); List msgList2 = new ArrayList<>(); @@ -1845,6 +1858,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(1000) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); consumerWillBeClose.close(); From fbf4cb71a3f3ed08786205dc5e60b810f3d62605 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 19 Apr 2024 21:51:10 +0800 Subject: [PATCH 42/55] [improve][offload] Apply autoSkipNonRecoverableData configuration to tiered storage (#22531) --- .../impl/BlobStoreBackedInputStreamImpl.java | 8 +++ .../impl/BlobStoreBackedReadHandleImpl.java | 13 ++++- .../impl/BlobStoreBackedReadHandleImplV2.java | 13 ++++- .../BlobStoreBackedInputStreamTest.java | 5 +- ...reManagedLedgerOffloaderStreamingTest.java | 54 +++++++++++++++++++ .../BlobStoreManagedLedgerOffloaderTest.java | 23 ++++++++ 6 files changed, 110 insertions(+), 6 deletions(-) diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 0dea46726f50a..6cb60e14984f9 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.blobstore.options.GetOptions; import org.slf4j.Logger; @@ -95,6 +96,9 @@ private boolean refillBufferIfNeeded() throws IOException { try { long startReadTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, key, new GetOptions().range(startRange, endRange)); + if (blob == null) { + throw new KeyNotFoundException(bucket, key, ""); + } versionCheck.check(key, blob); try (InputStream stream = blob.getPayload().openStream()) { @@ -121,6 +125,10 @@ private boolean refillBufferIfNeeded() throws IOException { if (null != this.offloaderStats) { this.offloaderStats.recordReadOffloadError(this.topicName); } + // If the blob is not found, the original exception is thrown and handled by the caller. + if (e instanceof KeyNotFoundException) { + throw e; + } throw new IOException("Error reading from BlobStore", e); } } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 5346be6a044c8..4f68f90370e6f 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -202,7 +203,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } catch (Throwable t) { log.error("Failed to read entries {} - {} from the offloader in ledger {}", firstEntry, lastEntry, ledgerId, t); - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } }); @@ -265,7 +270,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { int retryCount = 3; OffloadIndexBlock index = null; IOException lastException = null; @@ -278,6 +283,10 @@ public static ReadHandle open(ScheduledExecutorService executor, while (retryCount-- > 0) { long readIndexStartTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - readIndexStartTime, TimeUnit.NANOSECONDS); versionCheck.check(indexKey, blob); diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java index 53d96e08abf5e..502f475174cee 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -224,7 +225,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } } } catch (Throwable t) { - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } @@ -303,7 +308,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { List inputStreams = new LinkedList<>(); List indice = new LinkedList<>(); String topicName = TopicName.fromPersistenceNamingEncoding(managedLedgerName); @@ -313,6 +318,10 @@ public static ReadHandle open(ScheduledExecutorService executor, log.debug("open bucket: {} index key: {}", bucket, indexKey); long startTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - startTime, TimeUnit.NANOSECONDS); log.debug("indexKey blob: {} {}", indexKey, blob); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java index 775310925a1a3..3e5c4b609dfec 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java @@ -32,6 +32,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedInputStreamImpl; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.io.Payload; import org.jclouds.io.Payloads; @@ -142,8 +143,8 @@ public void testReadingFullObjectByBytes() throws Exception { assertStreamsMatchByBytes(toTest, toCompare); } - @Test(expectedExceptions = IOException.class) - public void testErrorOnRead() throws Exception { + @Test(expectedExceptions = KeyNotFoundException.class) + public void testNotFoundOnRead() throws Exception { BackedInputStream toTest = new BlobStoreBackedInputStreamImpl(blobStore, BUCKET, "doesn't exist", (key, md) -> {}, 1234, 1000); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java index 9056281a308f2..ad1529072f813 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java @@ -18,16 +18,19 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Random; import java.util.UUID; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; @@ -445,4 +448,55 @@ public void testInvalidEntryIds() throws Exception { } catch (Exception e) { } } + + @Test + public void testReadNotExistLedger() throws Exception { + LedgerOffloader offloader = getOffloader(new HashMap() {{ + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_SIZE_IN_BYTES, "1000"); + put(config.getKeys(TieredStorageConfiguration.METADATA_FIELD_MAX_BLOCK_SIZE).get(0), "5242880"); + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_ROLLOVER_TIME_SEC, "600"); + }}); + ManagedLedger ml = createMockManagedLedger(); + UUID uuid = UUID.randomUUID(); + long beginLedger = 0; + long beginEntry = 0; + + Map driverMeta = new HashMap() {{ + put(TieredStorageConfiguration.METADATA_FIELD_BUCKET, BUCKET); + }}; + OffloadHandle offloadHandle = offloader + .streamingOffload(ml, uuid, beginLedger, beginEntry, driverMeta).get(); + + // Segment should closed because size in bytes full + final LinkedList entries = new LinkedList<>(); + for (int i = 0; i < 10; i++) { + final byte[] data = new byte[100]; + random.nextBytes(data); + final EntryImpl entry = EntryImpl.create(0, i, data); + offloadHandle.offerEntry(entry); + entries.add(entry); + } + + final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); + assertEquals(offloadResult.endLedger, 0); + assertEquals(offloadResult.endEntry, 9); + final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); + contextBuilder.addOffloadSegment( + MLDataFormats.OffloadSegment.newBuilder() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9).build()); + + final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, uuid.toString()); + + try { + readHandle.read(0, 9); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index bb4cb286680f5..4419210c251f1 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -26,6 +27,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -601,4 +603,25 @@ public void testReadWithAClosedLedgerHandler() throws Exception { throw e; } } + + @Test + public void testReadNotExistLedger() throws Exception { + ReadHandle toWrite = buildReadHandle(DEFAULT_BLOCK_SIZE, 3); + LedgerOffloader offloader = getOffloader(); + + UUID uuid = UUID.randomUUID(); + offloader.offload(toWrite, uuid, new HashMap<>()).get(); + ReadHandle offloadRead = offloader.readOffloaded(toWrite.getId(), uuid, Collections.emptyMap()).get(); + assertEquals(offloadRead.getLastAddConfirmed(), toWrite.getLastAddConfirmed()); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, DataBlockUtils.dataBlockOffloadKey(toWrite.getId(), uuid)); + + try { + offloadRead.read(0, offloadRead.getLastAddConfirmed()); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } From 59daac64c210f539e733f883edad09d08333aa62 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 19 Apr 2024 10:30:55 -0700 Subject: [PATCH 43/55] [fix][broker] Fix broken topic policy implementation compatibility with old pulsar version (#22535) --- .../pulsar/broker/service/AbstractTopic.java | 52 +++++++++++-------- ...ternalClientConfigurationOverrideTest.java | 42 ++++++++++++++- 2 files changed, 72 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index e772486fcc6ea..44a4ca42cea46 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -220,13 +220,16 @@ protected void updateTopicPolicy(TopicPolicies data) { topicPolicies.getRetentionPolicies().updateTopicValue(data.getRetentionPolicies()); topicPolicies.getDispatcherPauseOnAckStatePersistentEnabled() .updateTopicValue(data.getDispatcherPauseOnAckStatePersistentEnabled()); - topicPolicies.getMaxSubscriptionsPerTopic().updateTopicValue(data.getMaxSubscriptionsPerTopic()); - topicPolicies.getMaxUnackedMessagesOnConsumer().updateTopicValue(data.getMaxUnackedMessagesOnConsumer()); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateTopicValue(normalizeValue(data.getMaxSubscriptionsPerTopic())); + topicPolicies.getMaxUnackedMessagesOnConsumer() + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnConsumer())); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateTopicValue(data.getMaxUnackedMessagesOnSubscription()); - topicPolicies.getMaxProducersPerTopic().updateTopicValue(data.getMaxProducerPerTopic()); - topicPolicies.getMaxConsumerPerTopic().updateTopicValue(data.getMaxConsumerPerTopic()); - topicPolicies.getMaxConsumersPerSubscription().updateTopicValue(data.getMaxConsumersPerSubscription()); + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnSubscription())); + topicPolicies.getMaxProducersPerTopic().updateTopicValue(normalizeValue(data.getMaxProducerPerTopic())); + topicPolicies.getMaxConsumerPerTopic().updateTopicValue(normalizeValue(data.getMaxConsumerPerTopic())); + topicPolicies.getMaxConsumersPerSubscription() + .updateTopicValue(normalizeValue(data.getMaxConsumersPerSubscription())); topicPolicies.getInactiveTopicPolicies().updateTopicValue(data.getInactiveTopicPolicies()); topicPolicies.getDeduplicationEnabled().updateTopicValue(data.getDeduplicationEnabled()); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateTopicValue( @@ -237,8 +240,8 @@ protected void updateTopicPolicy(TopicPolicies data) { Arrays.stream(BacklogQuota.BacklogQuotaType.values()).forEach(type -> this.topicPolicies.getBackLogQuotaMap().get(type).updateTopicValue( data.getBackLogQuotaMap() == null ? null : data.getBackLogQuotaMap().get(type.toString()))); - topicPolicies.getTopicMaxMessageSize().updateTopicValue(data.getMaxMessageSize()); - topicPolicies.getMessageTTLInSeconds().updateTopicValue(data.getMessageTTLInSeconds()); + topicPolicies.getTopicMaxMessageSize().updateTopicValue(normalizeValue(data.getMaxMessageSize())); + topicPolicies.getMessageTTLInSeconds().updateTopicValue(normalizeValue(data.getMessageTTLInSeconds())); topicPolicies.getPublishRate().updateTopicValue(PublishRate.normalize(data.getPublishRate())); topicPolicies.getDelayedDeliveryEnabled().updateTopicValue(data.getDelayedDeliveryEnabled()); topicPolicies.getReplicatorDispatchRate().updateTopicValue( @@ -268,15 +271,19 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { topicPolicies.getReplicationClusters().updateNamespaceValue( new ArrayList<>(CollectionUtils.emptyIfNull(namespacePolicies.replication_clusters))); topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_consumer); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_consumer)); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_subscription); - topicPolicies.getMessageTTLInSeconds().updateNamespaceValue(namespacePolicies.message_ttl_in_seconds); - topicPolicies.getMaxSubscriptionsPerTopic().updateNamespaceValue(namespacePolicies.max_subscriptions_per_topic); - topicPolicies.getMaxProducersPerTopic().updateNamespaceValue(namespacePolicies.max_producers_per_topic); - topicPolicies.getMaxConsumerPerTopic().updateNamespaceValue(namespacePolicies.max_consumers_per_topic); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_subscription)); + topicPolicies.getMessageTTLInSeconds() + .updateNamespaceValue(normalizeValue(namespacePolicies.message_ttl_in_seconds)); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_subscriptions_per_topic)); + topicPolicies.getMaxProducersPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_producers_per_topic)); + topicPolicies.getMaxConsumerPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_topic)); topicPolicies.getMaxConsumersPerSubscription() - .updateNamespaceValue(namespacePolicies.max_consumers_per_subscription); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_subscription)); topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(namespacePolicies.inactive_topic_policies); topicPolicies.getDeduplicationEnabled().updateNamespaceValue(namespacePolicies.deduplicationEnabled); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateNamespaceValue( @@ -312,6 +319,10 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { updateEntryFilters(); } + private Integer normalizeValue(Integer policyValue) { + return policyValue != null && policyValue < 0 ? null : policyValue; + } + private void updateNamespaceDispatchRate(Policies namespacePolicies, String cluster) { DispatchRateImpl dispatchRate = namespacePolicies.topicDispatchRate.get(cluster); if (dispatchRate == null) { @@ -370,12 +381,11 @@ private void updateTopicPolicyByBrokerConfig() { topicPolicies.getMaxConsumerPerTopic().updateBrokerValue(config.getMaxConsumersPerTopic()); topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(config.getMaxConsumersPerSubscription()); topicPolicies.getDeduplicationEnabled().updateBrokerValue(config.isBrokerDeduplicationEnabled()); - topicPolicies.getRetentionPolicies().updateBrokerValue(new RetentionPolicies( - config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); - topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateBrokerValue( - config.getBrokerDeduplicationSnapshotIntervalSeconds()); - topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); + topicPolicies.getRetentionPolicies().updateBrokerValue( + new RetentionPolicies(config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); + topicPolicies.getDeduplicationSnapshotIntervalSeconds() + .updateBrokerValue(config.getBrokerDeduplicationSnapshotIntervalSeconds()); + topicPolicies.getMaxUnackedMessagesOnConsumer().updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); topicPolicies.getMaxUnackedMessagesOnSubscription() .updateBrokerValue(config.getMaxUnackedMessagesPerSubscription()); //init backlogQuota diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java index 1b1b383e930e3..f33202c3c4033 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java @@ -18,17 +18,21 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.Policies; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; - +import lombok.Cleanup; import java.util.Optional; import java.util.Properties; @@ -112,4 +116,40 @@ public void testNamespaceServicePulsarClientConfiguration() { Assert.assertEquals(clientConf.getMemoryLimitBytes(), 100000); } + @Test + public void testOldNamespacePolicy() throws Exception { + + String ns = "prop/oldNsWithDefaultNonNullValues"; + String topic = "persistent://" + ns + "/t1"; + Policies policies = new Policies(); + policies.max_consumers_per_subscription = -1; + policies.max_consumers_per_topic = -1; + policies.max_producers_per_topic = -1; + policies.max_subscriptions_per_topic = -1; + policies.max_topics_per_namespace = -1; + policies.max_unacked_messages_per_consumer = -1; + policies.max_unacked_messages_per_subscription = -1; + admin.namespaces().createNamespace(ns, policies); + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic).create(); + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnSubscription().get(), + conf.getMaxUnackedMessagesPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumersPerSubscription().get(), + conf.getMaxConsumersPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumerPerTopic().get(), + conf.getMaxConsumersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxProducersPerTopic().get(), + conf.getMaxProducersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxSubscriptionsPerTopic().get(), + conf.getMaxSubscriptionsPerTopic()); + assertEquals(topicRef.topicPolicies.getTopicMaxMessageSize().get(), + conf.getMaxMessageSize()); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnConsumer().get(), + conf.getMaxUnackedMessagesPerConsumer()); + + + } } From 21647a1fc69ff46e65b6eaa37dd6d435e9f8eaef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 19 Apr 2024 19:12:34 -0700 Subject: [PATCH 44/55] [fix] Bump golang.org/x/net from 0.17.0 to 0.23.0 in /pulsar-function-go (#22540) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- pulsar-function-go/examples/go.mod | 8 ++++---- pulsar-function-go/examples/go.sum | 16 ++++++++-------- pulsar-function-go/go.mod | 8 ++++---- pulsar-function-go/go.sum | 16 ++++++++-------- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 31e1cc7769b92..59e695f5a33eb 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -42,11 +42,11 @@ require ( github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/stretchr/testify v1.8.4 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.17.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.15.0 // indirect - golang.org/x/term v0.15.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect golang.org/x/text v0.14.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index 5d2429673f028..85390cf32e59a 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= -golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index 1a0f2990f006f..bb5c18a4499e2 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -45,11 +45,11 @@ require ( github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.17.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.15.0 // indirect - golang.org/x/term v0.15.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect golang.org/x/text v0.14.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index 2cadeb1331f30..d840906772c56 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= -golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 4a887217d835629cafb393ddf331441b484d4e2c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 07:49:34 +0300 Subject: [PATCH 45/55] [improve][broker] Support X-Forwarded-For and HA Proxy Protocol for resolving original client IP of http/https requests (#22524) --- conf/broker.conf | 10 + conf/functions_worker.yml | 10 + conf/proxy.conf | 10 + conf/standalone.conf | 10 + conf/websocket.conf | 10 + pom.xml | 1 + .../pulsar/broker/ServiceConfiguration.java | 16 ++ .../broker/web/JettyRequestLogFactory.java | 195 +++++++++++++++++- pulsar-broker/pom.xml | 7 + .../apache/pulsar/broker/web/WebService.java | 37 +++- .../web/WebServiceOriginalClientIPTest.java | 155 ++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 3 +- .../pulsar/functions/worker/WorkerConfig.java | 16 ++ .../functions/worker/rest/WorkerServer.java | 38 +++- pulsar-proxy/pom.xml | 6 + .../proxy/server/ProxyConfiguration.java | 16 ++ .../proxy/server/ProxyServiceStarter.java | 31 ++- .../apache/pulsar/proxy/server/WebServer.java | 34 ++- .../server/ProxyOriginalClientIPTest.java | 157 ++++++++++++++ ...roxyServiceStarterDisableZeroCopyTest.java | 2 +- .../proxy/server/ProxyServiceStarterTest.java | 2 +- .../server/ProxyServiceTlsStarterTest.java | 2 +- pulsar-proxy/src/test/resources/log4j2.xml | 36 ++++ .../pulsar/websocket/service/ProxyServer.java | 39 +++- .../service/WebSocketProxyConfiguration.java | 14 ++ 25 files changed, 835 insertions(+), 22 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java create mode 100644 pulsar-proxy/src/test/resources/log4j2.xml diff --git a/conf/broker.conf b/conf/broker.conf index fd6bba0f45d2c..d482f77da7cb5 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -88,6 +88,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to config Netty Acceptor. Default is 1 numAcceptorThreads= diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 3871c74a88778..6f995576ebd64 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -27,6 +27,16 @@ workerHostname: localhost workerPort: 6750 workerPortTls: 6751 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled: false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor: false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses: null + # The Configuration metadata store url # Examples: # * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 diff --git a/conf/proxy.conf b/conf/proxy.conf index 5a9d433f39ceb..6e6c960e8009e 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -63,6 +63,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Enables zero-copy transport of data across network interfaces using the splice system call. # Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0. proxyZeroCopyModeEnabled=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 5c94d63817a12..b04e5ccefa640 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -51,6 +51,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() numIOThreads= diff --git a/conf/websocket.conf b/conf/websocket.conf index 490cff2722ee5..9051f3b590c8e 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -46,6 +46,16 @@ statusFilePath= # Hostname or IP address the service binds on, default is 0.0.0.0. bindAddress=0.0.0.0 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Name of the pulsar cluster to connect to clusterName= diff --git a/pom.xml b/pom.xml index c7fba94abd8ea..d4b14efc356ba 100644 --- a/pom.xml +++ b/pom.xml @@ -278,6 +278,7 @@ flexible messaging model and an intuitive client API. 1.5.4 5.4.0 2.33.2 + 1.0.3 0.6.1 diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 2b58cbc2d1178..156c83bd6960c 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -250,6 +250,22 @@ public class ServiceConfiguration implements PulsarConfiguration { + " when getting topic statistics data.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_SERVER, doc = "Number of threads to use for Netty Acceptor." diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java index e5daa5852b51f..fc88647eb49ea 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java @@ -18,9 +18,23 @@ */ package org.apache.pulsar.broker.web; +import java.net.InetSocketAddress; import java.util.TimeZone; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.eclipse.jetty.io.Connection; +import org.eclipse.jetty.io.EndPoint; +import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Response; +import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.Slf4jRequestLogWriter; +import org.eclipse.jetty.util.HostPort; +import org.eclipse.jetty.util.component.ContainerLifeCycle; /** * Class to standardize initialization of a Jetty request logger for all pulsar components. @@ -58,7 +72,184 @@ public class JettyRequestLogFactory { * Build a new Jetty request logger using the format defined in this class. * @return a request logger */ - public static CustomRequestLog createRequestLogger() { - return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + public static RequestLog createRequestLogger() { + return createRequestLogger(false, null); + } + + /** + * Build a new Jetty request logger using the format defined in this class. + * @param showDetailedAddresses whether to show detailed addresses and ports in logs + * @return a request logger + */ + public static RequestLog createRequestLogger(boolean showDetailedAddresses, Server server) { + if (!showDetailedAddresses) { + return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + } else { + return new OriginalClientIPRequestLog(server); + } + } + + /** + * Logs the original and real remote (client) and local (server) IP addresses + * when detailed addresses are enabled. + * Tracks the real addresses of remote and local using a registered Connection.Listener + * when detailed addresses are enabled. + * This is necessary when Proxy Protocol is used to pass the original client IP. + */ + @Slf4j + private static class OriginalClientIPRequestLog extends ContainerLifeCycle implements RequestLog { + private final ThreadLocal requestLogStringBuilder = ThreadLocal.withInitial(StringBuilder::new); + private final CustomRequestLog delegate; + private final Slf4jRequestLogWriter delegateLogWriter; + + OriginalClientIPRequestLog(Server server) { + delegate = new CustomRequestLog(this::write, LOG_FORMAT); + addBean(delegate); + delegateLogWriter = new Slf4jRequestLogWriter(); + addBean(delegateLogWriter); + if (server != null) { + for (Connector connector : server.getConnectors()) { + // adding the listener is only necessary for connectors that use ProxyConnectionFactory + if (connector.getDefaultConnectionFactory() instanceof ProxyConnectionFactory) { + connector.addBean(proxyProtocolOriginalEndpointListener); + } + } + } + } + + void write(String requestEntry) { + StringBuilder sb = requestLogStringBuilder.get(); + sb.setLength(0); + sb.append(requestEntry); + } + + @Override + public void log(Request request, Response response) { + delegate.log(request, response); + StringBuilder sb = requestLogStringBuilder.get(); + sb.append(" [R:"); + sb.append(request.getRemoteHost()); + sb.append(':'); + sb.append(request.getRemotePort()); + InetSocketAddress realRemoteAddress = lookupRealAddress(request.getHttpChannel().getRemoteAddress()); + if (realRemoteAddress != null) { + String realRemoteHost = HostPort.normalizeHost(realRemoteAddress.getHostString()); + int realRemotePort = realRemoteAddress.getPort(); + if (!realRemoteHost.equals(request.getRemoteHost()) || realRemotePort != request.getRemotePort()) { + sb.append(" via "); + sb.append(realRemoteHost); + sb.append(':'); + sb.append(realRemotePort); + } + } + sb.append("]->[L:"); + InetSocketAddress realLocalAddress = lookupRealAddress(request.getHttpChannel().getLocalAddress()); + if (realLocalAddress != null) { + String realLocalHost = HostPort.normalizeHost(realLocalAddress.getHostString()); + int realLocalPort = realLocalAddress.getPort(); + sb.append(realLocalHost); + sb.append(':'); + sb.append(realLocalPort); + if (!realLocalHost.equals(request.getLocalAddr()) || realLocalPort != request.getLocalPort()) { + sb.append(" dst "); + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + } else { + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + sb.append(']'); + try { + delegateLogWriter.write(sb.toString()); + } catch (Exception e) { + log.warn("Failed to write request log", e); + } + } + + private InetSocketAddress lookupRealAddress(InetSocketAddress socketAddress) { + if (socketAddress == null) { + return null; + } + if (proxyProtocolRealAddressMapping.isEmpty()) { + return socketAddress; + } + AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(socketAddress.getHostString(), + socketAddress.getPort())); + if (entry != null) { + return entry.realAddress; + } else { + return socketAddress; + } + } + + private final Connection.Listener proxyProtocolOriginalEndpointListener = + new ProxyProtocolOriginalEndpointListener(); + + private final ConcurrentHashMap proxyProtocolRealAddressMapping = + new ConcurrentHashMap<>(); + + // Use a record as key since InetSocketAddress hash code changes if the address gets resolved + record AddressKey(String hostString, int port) { + + } + + record AddressEntry(InetSocketAddress realAddress, AtomicInteger referenceCount) { + + } + + // Tracks the real addresses of remote and local when detailed addresses are enabled. + // This is necessary when Proxy Protocol is used to pass the original client IP. + // The Proxy Protocol implementation in Jetty wraps the original endpoint with a ProxyEndPoint + // and the real endpoint information isn't available in the request object. + // This listener is added to all connectors to track the real addresses of the client and server. + class ProxyProtocolOriginalEndpointListener implements Connection.Listener { + @Override + public void onOpened(Connection connection) { + handleConnection(connection, true); + } + + @Override + public void onClosed(Connection connection) { + handleConnection(connection, false); + } + + private void handleConnection(Connection connection, boolean increment) { + if (connection.getEndPoint() instanceof ProxyConnectionFactory.ProxyEndPoint) { + ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = + (ProxyConnectionFactory.ProxyEndPoint) connection.getEndPoint(); + EndPoint originalEndpoint = proxyEndPoint.unwrap(); + mapAddress(proxyEndPoint.getLocalAddress(), originalEndpoint.getLocalAddress(), increment); + mapAddress(proxyEndPoint.getRemoteAddress(), originalEndpoint.getRemoteAddress(), increment); + } + } + + private void mapAddress(InetSocketAddress current, InetSocketAddress real, boolean increment) { + // don't add the mapping if the current address is the same as the real address + if (real != null && current != null && current.equals(real)) { + return; + } + AddressKey key = new AddressKey(current.getHostString(), current.getPort()); + proxyProtocolRealAddressMapping.compute(key, (__, entry) -> { + if (entry == null) { + if (increment) { + entry = new AddressEntry(real, new AtomicInteger(1)); + } + } else { + if (increment) { + entry.referenceCount.incrementAndGet(); + } else { + if (entry.referenceCount.decrementAndGet() == 0) { + // remove the entry if the reference count drops to 0 + entry = null; + } + } + } + return entry; + }); + } + } } } diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index e15e024ea8158..3548877912199 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -176,6 +176,13 @@ test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + + io.streamnative.oxia oxia-testcontainers 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 8dc36e2917ed1..9a439268a8b4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -31,12 +31,18 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandler; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -103,9 +109,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { Optional port = config.getWebServicePort(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { - httpConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(port.get()); httpConnector.setHost(pulsar.getBindAddress()); connectors.add(httpConnector); @@ -144,7 +159,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - httpsConnector = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(tlsPort.get()); httpsConnector.setHost(pulsar.getBindAddress()); connectors.add(httpsConnector); @@ -284,7 +310,12 @@ public void addStaticResources(String basePath, String resourcePath) { public void start() throws PulsarServerException { try { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = pulsar.getConfiguration().getWebServiceLogDetailedAddresses() != null + ? pulsar.getConfiguration().getWebServiceLogDetailedAddresses() : + (pulsar.getConfiguration().isWebServiceHaProxyProtocolEnabled() + || pulsar.getConfiguration().isWebServiceTrustXForwardedFor()); + RequestLog requestLogger = JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server); + requestLogHandler.setRequestLog(requestLogger); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java new file mode 100644 index 0000000000000..7f7fa85bd3bb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.web; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class WebServiceOriginalClientIPTest extends MockedPulsarServiceBaseTest { + HttpClient httpClient; + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + conf.setWebServiceHaProxyProtocolEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("X-Forwarded-For", "11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromForwardedHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("Forwarded", "for=11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in Forwarded is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag(V2.Tag.Command.PROXY, null, V2.Tag.Protocol.STREAM, + // source IP and port + "99.22.33.44", 1234, + // destination IP and port + "5.4.3.1", 4321, + null)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP and destination IP passed in HA Proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:99.22.33.44:1234 via ") + && line.contains(" dst 5.4.3.1:4321]"))); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 38a57df80d57b..09a89702ee2ac 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -23,7 +23,8 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://logging.apache.org/log4j/2.0/config https://logging.apache.org/log4j/2.0/log4j-core.xsd"> - + + diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index ec0e620d0ae8b..036311ea13230 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -163,6 +163,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_WORKER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_WORKER, required = false, diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 2b3ea30121015..583d8ce558b08 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerStatsApiV2Resource; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -88,10 +95,21 @@ private void init() { server.addBean(new ConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); } + HttpConfiguration httpConfig = new HttpConfiguration(); + if (workerConfig.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (this.workerConfig.getWorkerPort() != null) { log.info("Configuring http server on port={}", this.workerConfig.getWorkerPort()); - httpConnector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(this.workerConfig.getWorkerPort()); connectors.add(httpConnector); } @@ -109,7 +127,10 @@ private void init() { workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer)); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = workerConfig.getWebServiceLogDetailedAddresses() != null + ? workerConfig.getWebServiceLogDetailedAddresses() : + (workerConfig.isWebServiceHaProxyProtocolEnabled() || workerConfig.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); @@ -161,7 +182,18 @@ private void init() { workerConfig.getTlsCertRefreshCheckDurationSec() ); } - httpsConnector = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); connectors.add(httpsConnector); } catch (Exception e) { diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 64ca301facf4d..a30e23b8d4781 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -209,6 +209,12 @@ ${wiremock.version} test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 39c8fb5e086fd..d65408748f432 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 @@ -268,6 +268,22 @@ public class ProxyConfiguration implements PulsarConfiguration { doc = "Enable or disable the proxy protocol.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(category = CATEGORY_SERVER, doc = "Enables zero-copy transport of data across network interfaces using the spice. " + "Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0.") diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 50a8e3ab7d753..10121e7f5d61d 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -35,6 +35,7 @@ import java.util.Collections; import java.util.Date; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import lombok.Getter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.core.util.datetime.FixedDateFormat; @@ -109,8 +110,15 @@ public class ProxyServiceStarter { private WebServer server; private WebSocketService webSocketService; private static boolean metricsInitialized; + private boolean embeddedMode; public ProxyServiceStarter(String[] args) throws Exception { + this(args, null, false); + } + + public ProxyServiceStarter(String[] args, Consumer proxyConfigurationCustomizer, + boolean embeddedMode) throws Exception { + this.embeddedMode = embeddedMode; try { DateFormat dateFormat = new SimpleDateFormat( FixedDateFormat.FixedFormat.ISO8601_OFFSET_DATE_TIME_HHMM.getPattern()); @@ -132,15 +140,26 @@ public ProxyServiceStarter(String[] args) throws Exception { CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); cmd.addCommand("proxy", commander); cmd.run(null); - System.exit(0); + if (embeddedMode) { + return; + } else { + System.exit(0); + } } } catch (Exception e) { commander.getErr().println(e); - System.exit(1); + if (embeddedMode) { + return; + } else { + System.exit(1); + } } // load config file config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class); + if (proxyConfigurationCustomizer != null) { + proxyConfigurationCustomizer.accept(config); + } if (!isBlank(zookeeperServers)) { // Use zookeeperServers from command line @@ -230,7 +249,9 @@ public void start() throws Exception { // create a web-service server = new WebServer(config, authenticationService); - Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + if (!embeddedMode) { + Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + } proxyService.start(); @@ -293,7 +314,9 @@ public void close() { } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!embeddedMode) { + LogManager.shutdown(); + } } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index b95bbcab08b11..478b911eb23cf 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -37,13 +37,18 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -93,12 +98,21 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication List connectors = new ArrayList<>(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setOutputBufferSize(config.getHttpOutputBufferSize()); httpConfig.setRequestHeaderSize(config.getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (config.getWebServicePort().isPresent()) { this.externalServicePort = config.getWebServicePort().get(); - connector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setHost(config.getBindAddress()); connector.setPort(externalServicePort); connectors.add(connector); @@ -133,7 +147,18 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectorTls.setHost(config.getBindAddress()); connectors.add(connectorTls); @@ -281,7 +306,10 @@ public int getExternalServicePort() { public void start() throws Exception { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = config.getWebServiceLogDetailedAddresses() != null + ? config.getWebServiceLogDetailedAddresses() : + (config.isWebServiceHaProxyProtocolEnabled() || config.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java new file mode 100644 index 0000000000000..b267439d47113 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { + static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; + HttpClient httpClient; + ProxyServiceStarter serviceStarter; + String webServiceUrl; + String webServiceUrlTls; + + @Override + @BeforeClass + protected void setup() throws Exception { + internalSetup(); + serviceStarter = new ProxyServiceStarter(ARGS, proxyConfig -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setWebServicePortTls(Optional.of(0)); + proxyConfig.setTlsEnabledWithBroker(false); + proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); + proxyConfig.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + proxyConfig.setWebServiceTrustXForwardedFor(true); + proxyConfig.setWebServiceHaProxyProtocolEnabled(true); + }, true); + serviceStarter.start(); + webServiceUrl = "http://localhost:" + serviceStarter.getServer().getListenPortHTTP().get(); + webServiceUrlTls = "https://localhost:" + serviceStarter.getServer().getListenPortHTTPS().get(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + internalCleanup(); + if (serviceStarter != null) { + serviceStarter.close(); + } + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + .header("X-Forwarded-For", "11.22.33.44") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag("99.22.33.44", 1234)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in HA proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java index 3e598a57277a2..937526629acf0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java @@ -27,7 +27,7 @@ public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterT @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index f263286125353..0b9b6f17d1254 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -54,7 +54,7 @@ public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 61718bbac3ab0..770424d93747c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -55,7 +55,7 @@ public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); diff --git a/pulsar-proxy/src/test/resources/log4j2.xml b/pulsar-proxy/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..261bd2edf6980 --- /dev/null +++ b/pulsar-proxy/src/test/resources/log4j2.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index 7aed43d056c67..bbb34a3e3f73d 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -73,10 +80,22 @@ public ProxyServer(WebSocketProxyConfiguration config) if (config.getMaxHttpServerConnections() > 0) { server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); } + + HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (config.getWebServicePort().isPresent()) { - connector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setPort(config.getWebServicePort().get()); connectors.add(connector); } @@ -111,7 +130,18 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectors.add(connectorTls); } catch (Exception e) { @@ -169,7 +199,10 @@ public void start() throws PulsarServerException { .map(ServerConnector.class::cast).map(ServerConnector::getPort).map(Object::toString) .collect(Collectors.joining(","))); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = conf.getWebServiceLogDetailedAddresses() != null + ? conf.getWebServiceLogDetailedAddresses() : + (conf.isWebServiceHaProxyProtocolEnabled() || conf.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 7acfd4a64ad35..3fcbcf4b21567 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -96,6 +96,20 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Hostname or IP address the service binds on, default is 0.0.0.0.") private String bindAddress = "0.0.0.0"; + @FieldContext(doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(doc = "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(doc = "Maximum size of a text message during parsing in WebSocket proxy") private int webSocketMaxTextFrameSize = 1024 * 1024; // --- Authentication --- From 3a0f908e80d0863920a1258362fd782e95fe8f17 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 22 Apr 2024 19:47:03 +0800 Subject: [PATCH 46/55] [improve][test] Add topic policy test for topic API (#22546) --- .../apache/pulsar/broker/admin/AuthZTest.java | 113 ++ .../pulsar/broker/admin/TopicAuthZTest.java | 1121 ++++++++++++----- .../admin/TransactionAndSchemaAuthZTest.java | 359 ++++++ 3 files changed, 1270 insertions(+), 323 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java new file mode 100644 index 0000000000000..a710a03970d06 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import static org.mockito.Mockito.doReturn; + +public class AuthZTest extends MockedPulsarStandalone { + + protected PulsarAdmin superUserAdmin; + + protected PulsarAdmin tenantManagerAdmin; + + protected AuthorizationService authorizationService; + + protected AuthorizationService orignalAuthorizationService; + + protected static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + protected static final String TENANT_ADMIN_TOKEN = Jwts.builder() + .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof TopicOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else if (operation instanceof NamespaceOperation) { + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + + + return execFlag; + } + + protected void createTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().createPartitionedTopic(topic, 2); + } else { + superUserAdmin.topics().createNonPartitionedTopic(topic); + } + } + + protected void deleteTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } else { + superUserAdmin.topics().delete(topic, true); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 3c0596d531f41..ad47ac74a8980 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,6 +20,7 @@ package org.apache.pulsar.broker.admin; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import lombok.Cleanup; import lombok.SneakyThrows; @@ -38,59 +39,48 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; +import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.security.MockedPulsarStandalone; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pulsar.broker.authorization.AuthorizationService; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.transaction.Transaction; -import org.apache.pulsar.common.naming.SystemTopicNames; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.metadata.api.MetadataStoreException; import org.mockito.Mockito; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import static org.mockito.Mockito.doReturn; @Test(groups = "broker-admin") -public class TopicAuthZTest extends MockedPulsarStandalone { - - private PulsarAdmin superUserAdmin; - - private PulsarAdmin tenantManagerAdmin; - - private AuthorizationService authorizationService; - - private AuthorizationService orignalAuthorizationService; - - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); - private static final String TENANT_ADMIN_TOKEN = Jwts.builder() - .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); +public class TopicAuthZTest extends AuthZTest { @SneakyThrows @BeforeClass(alwaysRun = true) public void setup() { configureTokenAuthentication(); configureDefaultAuthorization(); - enableTransaction(); start(); - createTransactionCoordinatorAssign(16); this.superUserAdmin =PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) @@ -103,16 +93,6 @@ public void setup() { .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); - superUserAdmin.tenants().createTenant("pulsar", tenantInfo); - superUserAdmin.namespaces().createNamespace("pulsar/system"); - } - - protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { - getPulsarService().getPulsarResources() - .getNamespaceResources() - .getPartitionedTopicResources() - .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, - new PartitionedTopicMetadata(numPartitionsOfTC)); } @SneakyThrows @@ -127,48 +107,28 @@ public void cleanup() { close(); } - @BeforeMethod - public void before() throws IllegalAccessException { - orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); - authorizationService = Mockito.spy(orignalAuthorizationService); - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - authorizationService, true); - } + private AtomicBoolean setAuthorizationPolicyOperationChecker(String role, Object policyName, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof PolicyOperation ) { - @AfterMethod - public void after() throws IllegalAccessException { - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - orignalAuthorizationService, true); - } + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); - private AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { - AtomicBoolean execFlag = new AtomicBoolean(false); - if (operation instanceof TopicOperation) { - Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); - } else if (operation instanceof NamespaceOperation) { Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); + String role_ = invocationOnMock.getArgument(4); + if (role.equals(role_)) { + PolicyName policyName_ = invocationOnMock.getArgument(1); + PolicyOperation operation_ = invocationOnMock.getArgument(2); + Assert.assertEquals(operation_, operation); + Assert.assertEquals(policyName_, policyName); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicPolicyOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any(), Mockito.any()); } else { throw new IllegalArgumentException(""); } - return execFlag; } @@ -1213,171 +1173,8 @@ public void testExpireMessageByPosition() { deleteTopic(topic, false); } - public enum OperationAuthType { - Lookup, - Produce, - Consume, - AdminOrSuperUser, - NOAuth - } - private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); - @FunctionalInterface - public interface ThrowingBiConsumer { - void accept(T t) throws PulsarAdminException; - } - @DataProvider(name = "authFunction") - public Object[][] authFunction () throws Exception { - String sub = "my-sub"; - createTopic(testTopic, false); - @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl(getPulsarService().getBrokerServiceUrl()) - .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) - .enableTransaction(true) - .build(); - @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); - - @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(testTopic) - .subscriptionName(sub) - .subscribe(); - - Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) - .build().get(); - MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); - - consumer.acknowledgeAsync(messageId, transaction).get(); - - return new Object[][]{ - // SCHEMA - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( - testTopic, 0), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( - testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, - SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.Produce - }, - // TODO: improve the authorization check for testCompatibility and deleteSchema - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( - testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( - testTopic), - OperationAuthType.AdminOrSuperUser - }, - - // TRANSACTION - - // Modify transaction coordinator - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .abortTransaction(transaction.getTxnID()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .scaleTransactionCoordinators(17), - OperationAuthType.AdminOrSuperUser - }, - // TODO: fix authorization check of check transaction coordinator stats. - // Check transaction coordinator stats - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorInternalStats(1, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorStats(), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionMetadata(transaction.getTxnID()), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .listTransactionCoordinators(), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactions(5, TimeUnit.SECONDS), - OperationAuthType.AdminOrSuperUser - }, - - // TODO: Check the authorization of the topic when get stats of TB or TP - // Check stats related to transaction buffer and transaction pending ack - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckInternalStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), - messageId.getEntryId(), null), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferInternalStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), - OperationAuthType.NOAuth - }, - }; - } @Test @SneakyThrows @@ -1410,82 +1207,7 @@ public void testSchemaCompatibility() { deleteTopic(topic, false); } - @Test(dataProvider = "authFunction") - public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) - throws Exception { - final String subject = UUID.randomUUID().toString(); - final String token = Jwts.builder() - .claim("sub", subject).signWith(SECRET_KEY).compact(); - - - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() - .serviceHttpUrl(getPulsarService().getWebServiceAddress()) - .authentication(new AuthenticationToken(token)) - .build(); - // test tenant manager - if (topicOpType != OperationAuthType.AdminOrSuperUser) { - adminConsumer.accept(tenantManagerAdmin); - } - - if (topicOpType != OperationAuthType.NOAuth) { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - - AtomicBoolean execFlag = null; - if (topicOpType == OperationAuthType.Lookup) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); - } else if (topicOpType == OperationAuthType.Produce) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); - } else if (topicOpType == OperationAuthType.Consume) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); - } - - for (AuthAction action : AuthAction.values()) { - superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); - - if (authActionMatchOperation(topicOpType, action)) { - adminConsumer.accept(subAdmin); - } else { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - superUserAdmin.topics().revokePermissions(testTopic, subject); - } - - if (execFlag != null) { - Assert.assertTrue(execFlag.get()); - } - - } - private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { - switch (operationAuthType) { - case Lookup -> { - if (AuthAction.consume == action || AuthAction.produce == action) { - return true; - } - } - case Consume -> { - if (AuthAction.consume == action) { - return true; - } - } - case Produce -> { - if (AuthAction.produce == action) { - return true; - } - } - case AdminOrSuperUser -> { - return false; - } - case NOAuth -> { - return true; - } - } - return false; - } @Test @SneakyThrows @@ -1507,8 +1229,10 @@ public void testGetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().getEntryFiltersPerTopic(topic, true); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.READ); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1553,8 +1277,10 @@ public void testSetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1656,19 +1382,768 @@ public void testShadowTopic() { deleteTopic(topic, false); } - private void createTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().createPartitionedTopic(topic, 2); - } else { - superUserAdmin.topics().createNonPartitionedTopic(topic); - } + @Test + @SneakyThrows + public void testList() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getList("public/default")); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicList("public/default")); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } - private void deleteTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().deletePartitionedTopic(topic, true); - } else { - superUserAdmin.topics().delete(topic, true); - } + @Test + @SneakyThrows + public void testPermissionsOnTopic() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().getPermissions(topic); + superUserAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + superUserAdmin.topics().revokePermissions(topic, subject); + + // test tenant manager + tenantManagerAdmin.topics().getPermissions(topic); + tenantManagerAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + tenantManagerAdmin.topics().revokePermissions(topic, subject); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPermissions(topic)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().revokePermissions(topic, subject)); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testOffloadPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setOffloadPolicies(topic, OffloadPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnConsumer() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplicationSnapshotInterval() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationSnapshotInterval(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testInactiveTopicPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setInactiveTopicPolicies(topic, new InactiveTopicPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDelayedDeliveryPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDelayedDeliveryPolicy(topic, DelayedDeliveryPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testAutoSubscriptionCreation() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getAutoSubscriptionCreation(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setAutoSubscriptionCreation(topic, AutoSubscriptionCreationOverride.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeAutoSubscriptionCreation(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscribeRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscribeRate(topic, new SubscribeRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscriptionTypesEnabled() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscriptionTypesEnabled(topic, new HashSet<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPublishRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPublishRate(topic, new PublishRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumersPerSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumersPerSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testCompactionThreshold() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setCompactionThreshold(topic, 20000)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxProducers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxProducers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicatorDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setReplicatorDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPersistence() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPersistence(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPersistence(topic, new PersistencePolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePersistence(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testRetention() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getRetention(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setRetention(topic, new RetentionPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeRetention(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplication() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationStatus(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMessageTTL() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMessageTTL(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMessageTTL(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMessageTTL(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testBacklogQuota() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getBacklogQuotaMap(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setBacklogQuota(topic, BacklogQuota.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeBacklogQuota(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicationClusters() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getReplicationClusters(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setReplicationClusters(topic, new ArrayList<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().removeReplicationClusters(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java new file mode 100644 index 0000000000000..1bca6f6e30835 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +@Test(groups = "broker-admin") +public class TransactionAndSchemaAuthZTest extends AuthZTest { + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + enableTransaction(); + start(); + createTransactionCoordinatorAssign(16); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + + superUserAdmin.tenants().createTenant("pulsar", tenantInfo); + superUserAdmin.namespaces().createNamespace("pulsar/system"); + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + } + if (tenantManagerAdmin != null) { + tenantManagerAdmin.close(); + } + close(); + } + + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { + getPulsarService().getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } + + public enum OperationAuthType { + Lookup, + Produce, + Consume, + AdminOrSuperUser, + NOAuth + } + + private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(T t) throws PulsarAdminException; + } + + @DataProvider(name = "authFunction") + public Object[][] authFunction () throws Exception { + String sub = "my-sub"; + createTopic(testTopic, false); + @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .enableTransaction(true) + .build(); + @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); + + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(testTopic) + .subscriptionName(sub) + .subscribe(); + + Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) + .build().get(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); + + consumer.acknowledgeAsync(messageId, transaction).get(); + + return new Object[][]{ + // SCHEMA + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( + testTopic, 0), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( + testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, + SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.Produce + }, + // TODO: improve the authorization check for testCompatibility and deleteSchema + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( + testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( + testTopic), + OperationAuthType.AdminOrSuperUser + }, + + // TRANSACTION + + // Modify transaction coordinator + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .abortTransaction(transaction.getTxnID()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .scaleTransactionCoordinators(17), + OperationAuthType.AdminOrSuperUser + }, + // TODO: fix authorization check of check transaction coordinator stats. + // Check transaction coordinator stats + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorInternalStats(1, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorStats(), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionMetadata(transaction.getTxnID()), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .listTransactionCoordinators(), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactions(5, TimeUnit.SECONDS), + OperationAuthType.AdminOrSuperUser + }, + + // TODO: Check the authorization of the topic when get stats of TB or TP + // Check stats related to transaction buffer and transaction pending ack + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckInternalStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), + messageId.getEntryId(), null), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferInternalStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), + OperationAuthType.NOAuth + }, + }; + } + + @Test(dataProvider = "authFunction") + public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) + throws Exception { + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test tenant manager + if (topicOpType != OperationAuthType.AdminOrSuperUser) { + adminConsumer.accept(tenantManagerAdmin); + } + + if (topicOpType != OperationAuthType.NOAuth) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + + AtomicBoolean execFlag = null; + if (topicOpType == OperationAuthType.Lookup) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); + } else if (topicOpType == OperationAuthType.Produce) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); + } else if (topicOpType == OperationAuthType.Consume) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); + + if (authActionMatchOperation(topicOpType, action)) { + adminConsumer.accept(subAdmin); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + superUserAdmin.topics().revokePermissions(testTopic, subject); + } + + if (execFlag != null) { + Assert.assertTrue(execFlag.get()); + } + + } + + private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { + switch (operationAuthType) { + case Lookup -> { + if (AuthAction.consume == action || AuthAction.produce == action) { + return true; + } + } + case Consume -> { + if (AuthAction.consume == action) { + return true; + } + } + case Produce -> { + if (AuthAction.produce == action) { + return true; + } + } + case AdminOrSuperUser -> { + return false; + } + case NOAuth -> { + return true; + } + } + return false; + } + +} From e81f37000ec212676c5daffa17faad8fc604ff77 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 18:13:45 +0300 Subject: [PATCH 47/55] [fix][offload] Increase file upload limit from 2048MiB to 4096MiB for GCP/GCS offloading (#22554) --- conf/broker.conf | 11 ++++++----- .../common/policies/data/OffloadPoliciesImpl.java | 7 ++++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index d482f77da7cb5..d97e3a5ef89ad 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1670,10 +1670,10 @@ s3ManagedLedgerOffloadBucket= # For Amazon S3 ledger offload, Alternative endpoint to connect to (useful for testing) s3ManagedLedgerOffloadServiceEndpoint= -# For Amazon S3 ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) +# For Amazon S3 ledger offload, Max block size in bytes. (64MiB by default, 5MiB minimum) s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864 -# For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default) +# For Amazon S3 ledger offload, Read buffer size in bytes (1MiB by default) s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage ledger offload, region where offload bucket is located. @@ -1683,10 +1683,11 @@ gcsManagedLedgerOffloadRegion= # For Google Cloud Storage ledger offload, Bucket to place offloaded ledger into gcsManagedLedgerOffloadBucket= -# For Google Cloud Storage ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) -gcsManagedLedgerOffloadMaxBlockSizeInBytes=67108864 +# For Google Cloud Storage ledger offload, Max block size in bytes. (128MiB by default, 5MiB minimum) +# Since JClouds limits the maximum number of blocks to 32, the maximum size of a ledger is 32 times the block size. +gcsManagedLedgerOffloadMaxBlockSizeInBytes=134217728 -# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MB by default) +# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MiB by default) gcsManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage, path to json file containing service account credentials. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index 51e181811c228..6c40aa3f2edd0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -79,8 +79,9 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { } } - public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MB - public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MB + public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MiB + public static final int DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES = 128 * 1024 * 1024; // 128MiB + public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MiB public static final int DEFAULT_OFFLOAD_MAX_THREADS = 2; public static final int DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS = 1; public static final String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders"; @@ -163,7 +164,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private String gcsManagedLedgerOffloadBucket = null; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) - private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; + private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) private Integer gcsManagedLedgerOffloadReadBufferSizeInBytes = DEFAULT_READ_BUFFER_SIZE_IN_BYTES; From 35599b7325347838203a92ca63b78d134b7864c2 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 23 Apr 2024 00:05:41 +0800 Subject: [PATCH 48/55] [fix][ml] Fix NPE of getValidPositionAfterSkippedEntries when recovering a terminated managed ledger (#22552) --- .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../mledger/impl/ManagedCursorTest.java | 61 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 698563ed7a1f2..70d3c1f74cab3 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 @@ -3701,7 +3701,7 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); // This means it has jumped to the last position if (nextLedgerId == null) { - if (currentLedgerEntries == 0) { + if (currentLedgerEntries == 0 && currentLedger != null) { return PositionImpl.get(currentLedger.getId(), 0); } return lastConfirmedEntry.getNext(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index c9bd64171c15a..4e3f8b7908438 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4695,5 +4695,66 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); } + @Test + public void testRecoverCursorWithTerminateManagedLedger() throws Exception { + String mlName = "my_test_ledger"; + String cursorName = "c1"; + + ManagedLedgerConfig config = new ManagedLedgerConfig(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(mlName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + + // Write some data. + Position p0 = ledger.addEntry("entry-0".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + + // Read message. + List entries = c1.readEntries(2); + assertEquals(entries.size(), 2); + assertEquals(entries.get(0).getPosition(), p0); + assertEquals(entries.get(1).getPosition(), p1); + entries.forEach(Entry::release); + + // Mark delete the last message. + c1.markDelete(p1); + Position markDeletedPosition = c1.getMarkDeletedPosition(); + Assert.assertEquals(markDeletedPosition, p1); + + // Terminate the managed ledger. + Position lastPosition = ledger.terminate(); + assertEquals(lastPosition, p1); + + // Close the ledger. + ledger.close(); + + // Reopen the ledger. + ledger = (ManagedLedgerImpl) factory.open(mlName, config); + BookKeeper mockBookKeeper = mock(BookKeeper.class); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, + cursorName); + + CompletableFuture recoverFuture = new CompletableFuture<>(); + // Recover the cursor. + cursor.recover(new VoidCallback() { + @Override + public void operationComplete() { + recoverFuture.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recoverFuture.completeExceptionally(exception); + } + }); + + recoverFuture.join(); + assertTrue(recoverFuture.isDone()); + assertFalse(recoverFuture.isCompletedExceptionally()); + + // Verify the cursor state. + assertEquals(cursor.getMarkDeletedPosition(), markDeletedPosition); + assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } From a037fa33eeeea6b0bc052c4aa960a55ca8bd0ca2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 19:38:11 +0300 Subject: [PATCH 49/55] [improve][misc] Upgrade to Bookkeeper 4.17.0 (#22551) --- .../server/src/assemble/LICENSE.bin.txt | 100 +++++++++--------- .../shell/src/assemble/LICENSE.bin.txt | 8 +- pom.xml | 6 +- 3 files changed, 57 insertions(+), 57 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4dc6e4341672c..93fd46d44b53f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -262,7 +262,7 @@ The Apache Software License, Version 2.0 - com.fasterxml.jackson.module-jackson-module-parameter-names-2.14.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar - * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.9.0.jar + * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson - com.google.code.gson-gson-2.8.9.jar @@ -356,34 +356,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar - - org.apache.bookkeeper-circe-checksum-4.16.5.jar - - org.apache.bookkeeper-cpu-affinity-4.16.5.jar - - org.apache.bookkeeper-statelib-4.16.5.jar - - org.apache.bookkeeper-stream-storage-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-common-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar - - org.apache.bookkeeper-stream-storage-server-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar - - org.apache.bookkeeper.http-http-server-4.16.5.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar - - org.apache.distributedlog-distributedlog-common-4.16.5.jar - - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.5.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar - - org.apache.bookkeeper-native-io-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-proto-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-server-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.17.0.jar + - org.apache.bookkeeper-circe-checksum-4.17.0.jar + - org.apache.bookkeeper-cpu-affinity-4.17.0.jar + - org.apache.bookkeeper-statelib-4.17.0.jar + - org.apache.bookkeeper-stream-storage-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-common-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.17.0.jar + - org.apache.bookkeeper-stream-storage-proto-4.17.0.jar + - org.apache.bookkeeper-stream-storage-server-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.17.0.jar + - org.apache.bookkeeper.http-http-server-4.17.0.jar + - org.apache.bookkeeper.http-vertx-http-server-4.17.0.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.0.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.0.jar + - org.apache.distributedlog-distributedlog-common-4.17.0.jar + - org.apache.distributedlog-distributedlog-core-4.17.0-tests.jar + - org.apache.distributedlog-distributedlog-core-4.17.0.jar + - org.apache.distributedlog-distributedlog-protocol-4.17.0.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.0.jar + - org.apache.bookkeeper-native-io-4.17.0.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -430,23 +430,23 @@ The Apache Software License, Version 2.0 - org.jetbrains.kotlin-kotlin-stdlib-jdk8-1.8.20.jar - org.jetbrains-annotations-13.0.jar * gRPC - - io.grpc-grpc-all-1.55.3.jar - - io.grpc-grpc-auth-1.55.3.jar - - io.grpc-grpc-context-1.55.3.jar - - io.grpc-grpc-core-1.55.3.jar - - io.grpc-grpc-netty-1.55.3.jar - - io.grpc-grpc-protobuf-1.55.3.jar - - io.grpc-grpc-protobuf-lite-1.55.3.jar - - io.grpc-grpc-stub-1.55.3.jar - - io.grpc-grpc-alts-1.55.3.jar - - io.grpc-grpc-api-1.55.3.jar - - io.grpc-grpc-grpclb-1.55.3.jar - - io.grpc-grpc-netty-shaded-1.55.3.jar - - io.grpc-grpc-services-1.55.3.jar - - io.grpc-grpc-xds-1.55.3.jar - - io.grpc-grpc-rls-1.55.3.jar - - io.grpc-grpc-servlet-1.55.3.jar - - io.grpc-grpc-servlet-jakarta-1.55.3.jar + - io.grpc-grpc-all-1.56.0.jar + - io.grpc-grpc-auth-1.56.0.jar + - io.grpc-grpc-context-1.56.0.jar + - io.grpc-grpc-core-1.56.0.jar + - io.grpc-grpc-netty-1.56.0.jar + - io.grpc-grpc-protobuf-1.56.0.jar + - io.grpc-grpc-protobuf-lite-1.56.0.jar + - io.grpc-grpc-stub-1.56.0.jar + - io.grpc-grpc-alts-1.56.0.jar + - io.grpc-grpc-api-1.56.0.jar + - io.grpc-grpc-grpclb-1.56.0.jar + - io.grpc-grpc-netty-shaded-1.56.0.jar + - io.grpc-grpc-services-1.56.0.jar + - io.grpc-grpc-xds-1.56.0.jar + - io.grpc-grpc-rls-1.56.0.jar + - io.grpc-grpc-servlet-1.56.0.jar + - io.grpc-grpc-servlet-jakarta-1.56.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar * OpenCensus @@ -504,8 +504,8 @@ The Apache Software License, Version 2.0 * 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 - - com.google.auto.value-auto-value-annotations-1.9.jar - - com.google.re2j-re2j-1.6.jar + - com.google.auto.value-auto-value-annotations-1.10.1.jar + - com.google.re2j-re2j-1.7.jar * Jetcd - io.etcd-jetcd-api-0.7.5.jar - io.etcd-jetcd-common-0.7.5.jar @@ -566,8 +566,8 @@ MIT License - com.auth0-jwks-rsa-0.22.0.jar Protocol Buffers License * Protocol Buffers - - com.google.protobuf-protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt - - com.google.protobuf-protobuf-java-util-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-util-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 069e61b89b55a..91d4643d9d4bc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -393,9 +393,9 @@ The Apache Software License, Version 2.0 - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - - bookkeeper-common-allocator-4.16.5.jar - - cpu-affinity-4.16.5.jar - - circe-checksum-4.16.5.jar + - bookkeeper-common-allocator-4.17.0.jar + - cpu-affinity-4.17.0.jar + - circe-checksum-4.17.0.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient @@ -429,7 +429,7 @@ MIT License Protocol Buffers License * Protocol Buffers - - protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/pom.xml b/pom.xml index d4b14efc356ba..168eddaf2fe1c 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.5 + 4.17.0 3.9.2 1.5.0 1.10.0 @@ -168,9 +168,9 @@ flexible messaging model and an intuitive client API. 0.5.0 1.14.12 1.17 - 3.19.6 + 3.22.3 ${protobuf3.version} - 1.55.3 + 1.56.0 1.41.0 0.26.0 ${grpc.version} From c72c135541e14043370836421cfef372b1d0a0ea Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 22 Apr 2024 14:15:36 -0700 Subject: [PATCH 50/55] [improve] Update Oxia client to 0.1.6 (#22525) --- .../licenses/LICENSE-Reactive-gRPC.txt | 29 +++++++++++++++++++ .../server/src/assemble/LICENSE.bin.txt | 10 ++++++- pom.xml | 3 +- pulsar-metadata/pom.xml | 1 - 4 files changed, 39 insertions(+), 4 deletions(-) create mode 100644 distribution/licenses/LICENSE-Reactive-gRPC.txt diff --git a/distribution/licenses/LICENSE-Reactive-gRPC.txt b/distribution/licenses/LICENSE-Reactive-gRPC.txt new file mode 100644 index 0000000000000..bc589401e7bdf --- /dev/null +++ b/distribution/licenses/LICENSE-Reactive-gRPC.txt @@ -0,0 +1,29 @@ +BSD 3-Clause License + +Copyright (c) 2019, Salesforce.com, Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 93fd46d44b53f..c5642503b25b0 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,7 +481,12 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-0.1.0-shaded.jar + - io.streamnative.oxia-oxia-client-0.1.6.jar + - io.streamnative.oxia-oxia-client-metrics-api-0.1.6.jar + * OpenHFT + - net.openhft-zero-allocation-hashing-0.16.jar + * Project reactor + - io.projectreactor-reactor-core-3.5.2.jar * Java JSON WebTokens - io.jsonwebtoken-jjwt-api-0.11.1.jar - io.jsonwebtoken-jjwt-impl-0.11.1.jar @@ -548,6 +553,9 @@ BSD 3-clause "New" or "Revised" License * JSR305 -- com.google.code.findbugs-jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt * JLine -- jline-jline-2.14.6.jar -- ../licenses/LICENSE-JLine.txt * JLine3 -- org.jline-jline-3.21.0.jar -- ../licenses/LICENSE-JLine.txt + * Reactive gRPC + - com.salesforce.servicelibs-reactive-grpc-common-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt + - com.salesforce.servicelibs-reactor-grpc-stub-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt BSD 2-Clause License * HdrHistogram -- org.hdrhistogram-HdrHistogram-2.1.9.jar -- ../licenses/LICENSE-HdrHistogram.txt diff --git a/pom.xml b/pom.xml index 168eddaf2fe1c..90b6c8cb8edf4 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.5 - 0.1.0 + 0.1.6 2.0 1.10.12 5.3.3 @@ -1193,7 +1193,6 @@ flexible messaging model and an intuitive client API. io.streamnative.oxia oxia-client ${oxia.version} - shaded io.streamnative.oxia diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 8600d0ea1919b..163a3058dc4bc 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -65,7 +65,6 @@ io.streamnative.oxia oxia-client - shaded From 49240522f543eea0e9307811c92b487eabe431d9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 23 Apr 2024 09:23:08 +0800 Subject: [PATCH 51/55] [fix] [broker] Part-1: Replicator can not created successfully due to an orphan replicator in the previous topic owner (#21946) --- .../broker/service/AbstractReplicator.java | 332 +++++++++++++----- .../pulsar/broker/service/BrokerService.java | 2 +- .../pulsar/broker/service/Replicator.java | 4 +- .../NonPersistentReplicator.java | 5 +- .../nonpersistent/NonPersistentTopic.java | 10 +- .../persistent/PersistentReplicator.java | 87 +++-- .../service/persistent/PersistentTopic.java | 31 +- .../service/AbstractReplicatorTest.java | 22 +- .../broker/service/OneWayReplicatorTest.java | 276 ++++++++++++++- .../service/OneWayReplicatorTestBase.java | 40 ++- .../broker/service/PersistentTopicTest.java | 6 +- .../pulsar/broker/service/ReplicatorTest.java | 11 +- 12 files changed, 656 insertions(+), 170 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 1b5b2824257b0..f34144deb0ab0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -18,16 +18,22 @@ */ package org.apache.pulsar.broker.service; +import com.google.common.annotations.VisibleForTesting; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Getter; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.Backoff; @@ -39,7 +45,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractReplicator { +public abstract class AbstractReplicator implements Replicator { protected final BrokerService brokerService; protected final String localTopicName; @@ -64,10 +70,31 @@ public abstract class AbstractReplicator { protected static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(AbstractReplicator.class, State.class, "state"); - private volatile State state = State.Stopped; - - protected enum State { - Stopped, Starting, Started, Stopping + @VisibleForTesting + @Getter + protected volatile State state = State.Disconnected; + + public enum State { + /** + * This enum has two mean meanings: + * Init: replicator is just created, has not been started now. + * Disconnected: the producer was closed after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is disconnected. + Disconnected, + // Trying to create a new internal producer. + Starting, + // The internal producer has started, and tries copy data. + Started, + /** + * The producer is closing after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is trying to disconnect. + Disconnecting, + // The replicator is in terminating. + Terminating, + // The replicator is never used again. Pulsar will create a new Replicator when enable replication again. + Terminated; } public AbstractReplicator(String localCluster, Topic localTopic, String remoteCluster, String remoteTopicName, @@ -96,16 +123,16 @@ public AbstractReplicator(String localCluster, Topic localTopic, String remoteCl .sendTimeout(0, TimeUnit.SECONDS) // .maxPendingMessages(producerQueueSize) // .producerName(getProducerName()); - STATE_UPDATER.set(this, State.Stopped); + STATE_UPDATER.set(this, State.Disconnected); } protected abstract String getProducerName(); - protected abstract void readEntries(org.apache.pulsar.client.api.Producer producer); + protected abstract void setProducerAndTriggerReadEntries(org.apache.pulsar.client.api.Producer producer); protected abstract Position getReplicatorReadPosition(); - protected abstract long getNumberOfEntriesInBacklog(); + public abstract long getNumberOfEntriesInBacklog(); protected abstract void disableReplicatorRead(); @@ -113,66 +140,121 @@ public String getRemoteCluster() { return remoteCluster; } - // This method needs to be synchronized with disconnects else if there is a disconnect followed by startProducer - // the end result can be disconnect. - public synchronized void startProducer() { - if (STATE_UPDATER.get(this) == State.Stopping) { - long waitTimeMs = backOff.next(); - if (log.isDebugEnabled()) { - log.debug( - "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", - replicatorId, waitTimeMs / 1000.0); - } - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); - return; - } - State state = STATE_UPDATER.get(this); - if (!STATE_UPDATER.compareAndSet(this, State.Stopped, State.Starting)) { - if (state == State.Started) { - // Already running + public void startProducer() { + // Guarantee only one task call "producerBuilder.createAsync()". + Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); + if (!setStartingRes.getLeft()) { + if (setStartingRes.getRight() == State.Starting) { + log.info("[{}] Skip the producer creation since other thread is doing starting, state : {}", + replicatorId, state); + } else if (setStartingRes.getRight() == State.Started) { + // Since the method "startProducer" will be called even if it is started, only print debug-level log. + if (log.isDebugEnabled()) { + log.debug("[{}] Replicator was already running. state: {}", replicatorId, state); + } + } else if (setStartingRes.getRight() == State.Disconnecting) { if (log.isDebugEnabled()) { - log.debug("[{}] Replicator was already running", replicatorId); + log.debug("[{}] Rep.producer is closing, delay to retry(wait the producer close success)." + + " state: {}", replicatorId, state); } + delayStartProducerAfterDisconnected(); } else { - log.info("[{}] Replicator already being started. Replicator state: {}", replicatorId, state); + /** {@link State.Terminating}, {@link State.Terminated}. **/ + log.info("[{}] Skip the producer creation since the replicator state is : {}", replicatorId, state); } - return; } log.info("[{}] Starting replicator", replicatorId); producerBuilder.createAsync().thenAccept(producer -> { - readEntries(producer); + setProducerAndTriggerReadEntries(producer); }).exceptionally(ex -> { - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopped)) { + Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { long waitTimeMs = backOff.next(); log.warn("[{}] Failed to create remote producer ({}), retrying in {} s", replicatorId, ex.getMessage(), waitTimeMs / 1000.0); - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } else { - log.warn("[{}] Failed to create remote producer. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectedRes.getRight() == State.Terminated) { + log.info("[{}] Skip to create producer, because it has been terminated, state is : {}", + replicatorId, state); + } else { + /** {@link State.Disconnected}, {@link State.Starting}, {@link State.Started} **/ + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other thread will try to create the producer again. so skipped current one task." + + " State is : {}", + replicatorId, state); + } } return null; }); + } + /*** + * The producer is disconnecting, delay to start the producer. + * If we start a producer immediately, we will get a conflict producer(same name producer) registered error. + */ + protected void delayStartProducerAfterDisconnected() { + long waitTimeMs = backOff.next(); + if (log.isDebugEnabled()) { + log.debug( + "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", + replicatorId, waitTimeMs / 1000.0); + } + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } - protected void checkTopicActiveAndRetryStartProducer() { - isLocalTopicActive().thenAccept(isTopicActive -> { - if (isTopicActive) { - startProducer(); + protected void scheduleCheckTopicActiveAndStartProducer(final long waitTimeMs) { + brokerService.executor().schedule(() -> { + if (state == State.Terminating || state == State.Terminated) { + log.info("[{}] Skip scheduled to start the producer since the replicator state is : {}", + replicatorId, state); + return; } - }).exceptionally(ex -> { - log.warn("[{}] Stop retry to create producer due to topic load fail. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); - return null; - }); + CompletableFuture> topicFuture = brokerService.getTopics().get(localTopicName); + if (topicFuture == null) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed successfully." + + " And trigger a terminate.", replicatorId); + terminate(); + return; + } + topicFuture.thenAccept(optional -> { + if (optional.isEmpty()) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + if (optional.get() != localTopic) { + // Topic closed and created a new one, current replicator is outdated. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + Replicator replicator = localTopic.getReplicators().get(remoteCluster); + if (replicator != AbstractReplicator.this) { + // Current replicator has been closed, and created a new one. + log.info("[{}] Skip scheduled to start the producer since a new replicator has instead current" + + " one. And trigger a terminate.", replicatorId); + terminate(); + return; + } + startProducer(); + }).exceptionally(ex -> { + log.warn("[{}] [{}] Stop retry to create producer due to unknown error(topic create failed), and" + + " trigger a terminate. Replicator state: {}", + localTopicName, replicatorId, STATE_UPDATER.get(this), ex); + terminate(); + return null; + }); + }, waitTimeMs, TimeUnit.MILLISECONDS); } protected CompletableFuture isLocalTopicActive() { @@ -188,58 +270,130 @@ protected CompletableFuture isLocalTopicActive() { }, brokerService.executor()); } - protected synchronized CompletableFuture closeProducerAsync() { - if (producer == null) { - STATE_UPDATER.set(this, State.Stopped); + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + public CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer) { + long backlog = getNumberOfEntriesInBacklog(); + if (failIfHasBacklog && backlog > 0) { + CompletableFuture disconnectFuture = new CompletableFuture<>(); + disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); + if (log.isDebugEnabled()) { + log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); + } + return disconnectFuture; + } + log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, + getReplicatorReadPosition(), backlog); + return closeProducerAsync(closeTheStartingProducer); + } + + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + protected CompletableFuture closeProducerAsync(boolean closeTheStartingProducer) { + Pair setDisconnectingRes = compareSetAndGetState(State.Started, State.Disconnecting); + if (!setDisconnectingRes.getLeft()) { + if (setDisconnectingRes.getRight() == State.Starting) { + if (closeTheStartingProducer) { + /** + * Delay retry(wait for the start producer task is finish). + * Note: If the producer always start fail, the start producer task will always retry until the + * state changed to {@link State.Terminated}. + * Nit: The better solution is creating a {@link CompletableFuture} to trace the in-progress + * creation and call "inProgressCreationFuture.thenApply(closeProducer())". + */ + long waitTimeMs = backOff.next(); + brokerService.executor().schedule(() -> closeProducerAsync(true), + waitTimeMs, TimeUnit.MILLISECONDS); + } else { + log.info("[{}] Skip current producer closing since the previous producer has been closed," + + " and trying start a new one, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } + } else if (setDisconnectingRes.getRight() == State.Disconnected + || setDisconnectingRes.getRight() == State.Disconnecting) { + log.info("[{}] Skip current producer closing since other thread did closing, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } else if (setDisconnectingRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip current producer closing since other thread is doing termination, state : {}", + replicatorId, state); + } + log.info("[{}] Skip current termination since other thread is doing close producer or termination," + + " state : {}", replicatorId, state); return CompletableFuture.completedFuture(null); } - CompletableFuture future = producer.closeAsync(); + + // Close producer and update state. + return doCloseProducerAsync(producer, () -> { + Pair setDisconnectedRes = compareSetAndGetState(State.Disconnecting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { + this.producer = null; + // deactivate further read + disableReplicatorRead(); + return; + } + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip setting state to terminated because it was terminated, state : {}", + replicatorId, state); + } else { + // Since only one task can call "doCloseProducerAsync(producer, action)", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other task has change the state to terminated. so skipped current one task." + + " State is : {}", + replicatorId, state); + } + }); + } + + protected CompletableFuture doCloseProducerAsync(Producer producer, Runnable actionAfterClosed) { + CompletableFuture future = + producer == null ? CompletableFuture.completedFuture(null) : producer.closeAsync(); return future.thenRun(() -> { - STATE_UPDATER.set(this, State.Stopped); - this.producer = null; - // deactivate further read - disableReplicatorRead(); + actionAfterClosed.run(); }).exceptionally(ex -> { long waitTimeMs = backOff.next(); log.warn( - "[{}] Exception: '{}' occurred while trying to close the producer." - + " retrying again in {} s", - replicatorId, ex.getMessage(), waitTimeMs / 1000.0); + "[{}] Exception: '{}' occurred while trying to close the producer. Replicator state: {}." + + " Retrying again in {} s.", + replicatorId, ex.getMessage(), state, waitTimeMs / 1000.0); // BackOff before retrying - brokerService.executor().schedule(this::closeProducerAsync, waitTimeMs, TimeUnit.MILLISECONDS); + brokerService.executor().schedule(() -> doCloseProducerAsync(producer, actionAfterClosed), + waitTimeMs, TimeUnit.MILLISECONDS); return null; }); } - - public CompletableFuture disconnect() { - return disconnect(false); + public CompletableFuture terminate() { + if (!tryChangeStatusToTerminating()) { + log.info("[{}] Skip current termination since other thread is doing termination, state : {}", replicatorId, + state); + return CompletableFuture.completedFuture(null); + } + return doCloseProducerAsync(producer, () -> { + STATE_UPDATER.set(this, State.Terminated); + this.producer = null; + // set the cursor as inactive. + disableReplicatorRead(); + }); } - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - if (failIfHasBacklog && getNumberOfEntriesInBacklog() > 0) { - CompletableFuture disconnectFuture = new CompletableFuture<>(); - disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); - if (log.isDebugEnabled()) { - log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); - } - return disconnectFuture; + protected boolean tryChangeStatusToTerminating() { + if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Terminating)){ + return true; } - - if (STATE_UPDATER.get(this) == State.Stopping) { - // Do nothing since the all "STATE_UPDATER.set(this, Stopping)" instructions are followed by - // closeProducerAsync() - // which will at some point change the state to stopped - return CompletableFuture.completedFuture(null); + if (STATE_UPDATER.compareAndSet(this, State.Started, State.Terminating)){ + return true; } - - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopping) - || STATE_UPDATER.compareAndSet(this, State.Started, State.Stopping)) { - log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, - getReplicatorReadPosition(), getNumberOfEntriesInBacklog()); + if (STATE_UPDATER.compareAndSet(this, State.Disconnecting, State.Terminating)){ + return true; } - - return closeProducerAsync(); + if (STATE_UPDATER.compareAndSet(this, State.Disconnected, State.Terminating)) { + return true; + } + return false; } public CompletableFuture remove() { @@ -300,4 +454,18 @@ public static CompletableFuture validatePartitionedTopicAsync(String topic public State getState() { return state; } + + protected ImmutablePair compareSetAndGetState(State expect, State update) { + State original1 = state; + if (STATE_UPDATER.compareAndSet(this, expect, update)) { + return ImmutablePair.of(true, expect); + } + State original2 = state; + // Maybe the value changed more than once even if "original1 == original2", but the probability is very small, + // so let's ignore this case for prevent using a lock. + if (original1 == original2) { + return ImmutablePair.of(false, original1); + } + return compareSetAndGetState(expect, update); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 249008bad91ad..295a9a2954126 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -744,7 +744,7 @@ public CompletableFuture closeAndRemoveReplicationClient(String clusterNam if (ot.isPresent()) { Replicator r = ot.get().getReplicators().get(clusterName); if (r != null && r.isConnected()) { - r.disconnect(false).whenComplete((v, e) -> f.complete(null)); + r.terminate().whenComplete((v, e) -> f.complete(null)); return; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 482fa2cbd2300..8130b855b4e4a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -29,9 +29,9 @@ public interface Replicator { ReplicatorStatsImpl getStats(); - CompletableFuture disconnect(); + CompletableFuture terminate(); - CompletableFuture disconnect(boolean b); + CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer); void updateRates(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 087c5f932008f..51509f3818a28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -67,7 +67,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { this.producer = (ProducerImpl) producer; if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { @@ -78,8 +78,7 @@ protected void readEntries(Producer producer) { "[{}] Replicator was stopped while creating the producer." + " Closing it. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + doCloseProducerAsync(producer, () -> {}); return; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 9a3a0a7d83d50..586fcd76151e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -420,7 +420,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c CompletableFuture closeClientFuture = new CompletableFuture<>(); if (closeIfClientsConnected) { List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); FutureUtil.waitForAll(futures).thenRun(() -> { @@ -523,7 +523,7 @@ public CompletableFuture close( List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -582,7 +582,7 @@ public CompletableFuture close( public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } @@ -663,7 +663,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = NonPersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - replicators.get(remoteCluster).disconnect().thenRun(() -> { + replicators.get(remoteCluster).terminate().thenRun(() -> { log.info("[{}] Successfully removed replicator {}", name, remoteCluster); replicators.remove(remoteCluster); @@ -1032,7 +1032,7 @@ private CompletableFuture disconnectReplicators() { List> futures = new ArrayList<>(); ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); }); return FutureUtil.waitForAll(futures); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 754d25b8b0ab4..5e1cc4a936a75 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Started; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Starting; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminated; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminating; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; @@ -26,7 +30,6 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -43,10 +46,10 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.AbstractReplicator; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; @@ -134,30 +137,44 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man } @Override - protected void readEntries(Producer producer) { - // Rewind the cursor to be sure to read again all non-acked messages sent while restarting + protected void setProducerAndTriggerReadEntries(Producer producer) { + // Rewind the cursor to be sure to read again all non-acked messages sent while restarting. cursor.rewind(); - cursor.cancelPendingReadRequest(); - HAVE_PENDING_READ_UPDATER.set(this, FALSE); - this.producer = (ProducerImpl) producer; - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { - log.info("[{}] Created replicator producer", replicatorId); + /** + * 1. Try change state to {@link Started}. + * 2. Atoms modify multiple properties if change state success, to avoid another thread get a null value + * producer when the state is {@link Started}. + */ + Pair changeStateRes; + changeStateRes = compareSetAndGetState(Starting, Started); + if (changeStateRes.getLeft()) { + this.producer = (ProducerImpl) producer; + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + // Trigger a new read. + log.info("[{}] Created replicator producer, Replicator state: {}", replicatorId, state); backOff.reset(); - // activate cursor: so, entries can be cached + // activate cursor: so, entries can be cached. this.cursor.setActive(); // read entries readMoreEntries(); } else { - log.info( - "[{}] Replicator was stopped while creating the producer." - + " Closing it. Replicator state: {}", - replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + if (changeStateRes.getRight() == Started) { + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Replicator was already started by another thread while creating the producer." + + " Closing the producer newly created. Replicator state: {}", replicatorId, state); + } else if (changeStateRes.getRight() == Terminating || changeStateRes.getRight() == Terminated) { + log.info("[{}] Replicator was terminated, so close the producer. Replicator state: {}", + replicatorId, state); + } else { + log.error("[{}] Replicator state is not expected, so close the producer. Replicator state: {}", + replicatorId, changeStateRes.getRight()); + } + // Close the producer if change the state fail. + doCloseProducerAsync(producer, () -> {}); } - } @Override @@ -420,8 +437,8 @@ public CompletableFuture getFuture() { @Override public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - if (STATE_UPDATER.get(this) != State.Started) { - log.info("[{}] Replicator was stopped while reading entries." + if (state != Started) { + log.info("[{}] Replicator was disconnected while reading entries." + " Stop reading. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); return; @@ -436,8 +453,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}] Error reading entries because replicator is" + " already deleted and cursor is already closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } else if (!(exception instanceof TooManyRequestsException)) { log.error("[{}] Error reading entries at {}. Retrying to read in {}s. ({})", @@ -555,8 +572,8 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { if (exception instanceof CursorAlreadyClosedException) { log.error("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + " closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } if (ctx instanceof PositionImpl) { @@ -675,30 +692,6 @@ protected void checkReplicatedSubscriptionMarker(Position position, MessageImpl< } } - @Override - public CompletableFuture disconnect() { - return disconnect(false); - } - - @Override - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - final CompletableFuture future = new CompletableFuture<>(); - - super.disconnect(failIfHasBacklog).thenRun(() -> { - dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); - future.complete(null); - }).exceptionally(ex -> { - Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex); - if (!(t instanceof TopicBusyException)) { - log.error("[{}] Failed to close dispatch rate limiter: {}", replicatorId, ex.getMessage()); - } - future.completeExceptionally(t); - return null; - }); - - return future; - } - @Override public boolean isConnected() { ProducerImpl producer = this.producer; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 936091edce557..9d6855962ced6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -833,15 +833,15 @@ public CompletableFuture startReplProducers() { public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } private synchronized CompletableFuture closeReplProducersIfNoBacklog() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true))); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true))); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true, true))); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true, true))); return FutureUtil.waitForAll(closeFutures); } @@ -1423,8 +1423,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, List> futures = new ArrayList<>(); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); if (closeIfClientsConnected) { - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); } FutureUtil.waitForAll(futures).thenRunAsync(() -> { @@ -1565,8 +1565,8 @@ public CompletableFuture close( List> futures = new ArrayList<>(); futures.add(transactionBuffer.closeAsync()); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -1942,7 +1942,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::disconnect) + Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::terminate) .orElse(CompletableFuture.completedFuture(null)).thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2014,7 +2014,7 @@ CompletableFuture removeShadowReplicator(String shadowTopic) { log.info("[{}] Removing shadow topic replicator to {}", topic, shadowTopic); final CompletableFuture future = new CompletableFuture<>(); String name = ShadowReplicator.getShadowReplicatorName(replicatorPrefix, shadowTopic); - shadowReplicators.get(shadowTopic).disconnect().thenRun(() -> { + shadowReplicators.get(shadowTopic).terminate().thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2898,7 +2898,7 @@ private CompletableFuture checkAndDisconnectReplicators() { ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { if (replicator.getNumberOfEntriesInBacklog() <= 0) { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); } }); return FutureUtil.waitForAll(futures); @@ -2949,6 +2949,15 @@ public void checkGC() { log.debug("[{}] Global topic inactive for {} seconds, closing repl producers.", topic, maxInactiveDurationInSec); } + /** + * There is a race condition that may cause a NPE: + * - task 1: a callback of "replicator.cursor.asyncRead" will trigger a replication. + * - task 2: "closeReplProducersIfNoBacklog" called by current thread will make the variable + * "replicator.producer" to a null value. + * Race condition: task 1 will get a NPE when it tries to send messages using the variable + * "replicator.producer", because task 2 will set this variable to "null". + * TODO Create a seperated PR to fix it. + */ closeReplProducersIfNoBacklog().thenRun(() -> { if (hasRemoteProducers()) { if (log.isDebugEnabled()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 8699c73246830..7aebf20896c2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -94,7 +95,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { final ReplicatorInTest replicator = new ReplicatorInTest(localCluster, localTopic, remoteCluster, topicName, replicatorPrefix, broker, remoteClient); replicator.startProducer(); - replicator.disconnect(); + replicator.terminate(); // Verify task will done. Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { @@ -129,7 +130,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { } @@ -139,7 +140,22 @@ protected Position getReplicatorReadPosition() { } @Override - protected long getNumberOfEntriesInBacklog() { + public ReplicatorStatsImpl getStats() { + return null; + } + + @Override + public void updateRates() { + + } + + @Override + public boolean isConnected() { + return false; + } + + @Override + public long getNumberOfEntriesInBacklog() { return 0; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 1accd04f4918c..f9184f2288f52 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -18,28 +18,56 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +@Slf4j @Test(groups = "broker") public class OneWayReplicatorTest extends OneWayReplicatorTestBase { @@ -78,7 +106,7 @@ private ProducerImpl overrideProducerForReplicator(AbstractReplicator replicator return originalValue; } - @Test + @Test(timeOut = 45 * 1000) public void testReplicatorProducerStatInTopic() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); final String subscribeName = "subscribe_1"; @@ -104,7 +132,7 @@ public void testReplicatorProducerStatInTopic() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testCreateRemoteConsumerFirst() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); @@ -124,29 +152,257 @@ public void testCreateRemoteConsumerFirst() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); admin1.topics().createNonPartitionedTopic(topicName); // Wait for replicator started. waitReplicatorStarted(topicName); - PersistentTopic persistentTopic = + PersistentTopic topic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); - PersistentReplicator replicator = - (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + PersistentReplicator replicator1 = + (PersistentReplicator) topic1.getReplicators().values().iterator().next(); // Mock an error when calling "replicator.disconnect()" - ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); - Mockito.when(mockProducer.closeAsync()).thenReturn(CompletableFuture.failedFuture(new Exception("mocked ex"))); - ProducerImpl originalProducer = overrideProducerForReplicator(replicator, mockProducer); + AtomicBoolean closeFailed = new AtomicBoolean(true); + final ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); + final AtomicReference originalProducer1 = new AtomicReference(); + doAnswer(invocation -> { + if (closeFailed.get()) { + return CompletableFuture.failedFuture(new Exception("mocked ex")); + } else { + return originalProducer1.get().closeAsync(); + } + }).when(mockProducer).closeAsync(); + originalProducer1.set(overrideProducerForReplicator(replicator1, mockProducer)); // Verify: since the "replicator.producer.closeAsync()" will retry after it failed, the topic unload should be // successful. admin1.topics().unload(topicName); // Verify: After "replicator.producer.closeAsync()" retry again, the "replicator.producer" will be closed // successful. - overrideProducerForReplicator(replicator, originalProducer); + closeFailed.set(false); + AtomicReference topic2 = new AtomicReference(); + AtomicReference replicator2 = new AtomicReference(); Awaitility.await().untilAsserted(() -> { + topic2.set((PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get()); + replicator2.set((PersistentReplicator) topic2.get().getReplicators().values().iterator().next()); + // It is a new Topic after reloading. + assertNotEquals(topic2.get(), topic1); + assertNotEquals(replicator2.get(), replicator1); + }); + Awaitility.await().untilAsserted(() -> { + // Old replicator should be closed. + Assert.assertFalse(replicator1.isConnected()); + Assert.assertFalse(originalProducer1.get().isConnected()); + // New replicator should be connected. + Assert.assertTrue(replicator2.get().isConnected()); + }); + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } + + private void injectMockReplicatorProducerBuilder( + BiFunction producerDecorator) + throws Exception { + String cluster2 = pulsar2.getConfig().getClusterName(); + BrokerService brokerService = pulsar1.getBrokerService(); + // Wait for the internal client created. + final String topicNameTriggerInternalClientCreate = + BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicNameTriggerInternalClientCreate); + waitReplicatorStarted(topicNameTriggerInternalClientCreate); + cleanupTopics(() -> { + admin1.topics().delete(topicNameTriggerInternalClientCreate); + admin2.topics().delete(topicNameTriggerInternalClientCreate); + }); + + // Inject spy client. + ConcurrentOpenHashMap + replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients"); + PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2); + PulsarClient spyClient = spy(internalClient); + replicationClients.put(cluster2, spyClient); + + // Inject producer decorator. + doAnswer(invocation -> { + Schema schema = (Schema) invocation.getArguments()[0]; + ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) internalClient.newProducer(schema); + ProducerBuilder spyProducerBuilder = spy(producerBuilder); + doAnswer(ignore -> { + CompletableFuture producerFuture = new CompletableFuture<>(); + producerBuilder.createAsync().whenComplete((p, t) -> { + if (t != null) { + producerFuture.completeExceptionally(t); + return; + } + ProducerImpl pImpl = (ProducerImpl) p; + new FastThreadLocalThread(() -> { + try { + ProducerImpl newProducer = producerDecorator.apply(producerBuilder.getConf(), pImpl); + producerFuture.complete(newProducer); + } catch (Exception ex) { + producerFuture.completeExceptionally(ex); + } + }).start(); + }); + + return producerFuture; + }).when(spyProducerBuilder).createAsync(); + return spyProducerBuilder; + }).when(spyClient).newProducer(any(Schema.class)); + } + + private SpyCursor spyCursor(PersistentTopic persistentTopic, String cursorName) throws Exception { + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(cursorName); + ManagedCursorImpl spyCursor = spy(cursor); + // remove cursor. + ml.getCursors().removeCursor(cursorName); + ml.deactivateCursor(cursor); + // Add the spy one. addCursor(ManagedCursorImpl cursor) + Method m = ManagedLedgerImpl.class.getDeclaredMethod("addCursor", new Class[]{ManagedCursorImpl.class}); + m.setAccessible(true); + m.invoke(ml, new Object[]{spyCursor}); + return new SpyCursor(cursor, spyCursor); + } + + @Data + @AllArgsConstructor + static class SpyCursor { + ManagedCursorImpl original; + ManagedCursorImpl spy; + } + + private CursorCloseSignal makeCursorClosingDelay(SpyCursor spyCursor) throws Exception { + CountDownLatch startCloseSignal = new CountDownLatch(1); + CountDownLatch startCallbackSignal = new CountDownLatch(1); + doAnswer(invocation -> { + AsyncCallbacks.CloseCallback originalCallback = (AsyncCallbacks.CloseCallback) invocation.getArguments()[0]; + Object ctx = invocation.getArguments()[1]; + AsyncCallbacks.CloseCallback newCallback = new AsyncCallbacks.CloseCallback() { + @Override + public void closeComplete(Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeComplete(ctx); + } + }).start(); + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeFailed(exception, ctx); + } + }).start(); + } + }; + startCloseSignal.await(); + spyCursor.original.asyncClose(newCallback, ctx); + return null; + }).when(spyCursor.spy).asyncClose(any(AsyncCallbacks.CloseCallback.class), any()); + return new CursorCloseSignal(startCloseSignal, startCallbackSignal); + } + + @AllArgsConstructor + static class CursorCloseSignal { + CountDownLatch startCloseSignal; + CountDownLatch startCallbackSignal; + + void startClose() { + startCloseSignal.countDown(); + } + + void startCallback() { + startCallbackSignal.countDown(); + } + } + + /** + * See the description and execution flow: https://github.com/apache/pulsar/pull/21946. + * Steps: + * - Create topic, but the internal producer of Replicator created failed. + * - Unload bundle, the Replicator will be closed, but the internal producer creation retry has not executed yet. + * - The internal producer creation retry execute successfully, the "repl.cursor" has not been closed yet. + * - The topic is wholly closed. + * - Verify: the delayed created internal producer will be closed. + */ + @Test(timeOut = 120 * 1000) + public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + // Inject an error for "replicator.producer" creation. + // The delay time of next retry to create producer is below: + // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... + // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final int failTimes = 6; + injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (createProducerCounter.incrementAndGet() > failTimes) { + return originalProducer; + } + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + }); + + // Create topic. + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + PersistentReplicator replicator = + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + // Since we inject a producer creation error, the replicator can not start successfully. + assertFalse(replicator.isConnected()); + + // Stuck the closing of the cursor("pulsar.repl"), until the internal producer of the replicator started. + SpyCursor spyCursor = + spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName()); + CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor); + + // Unload bundle: call "topic.close(false)". + // Stuck start new producer, until the state of replicator change to Stopped. + // The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully. + Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + assertTrue(createProducerCounter.get() >= failTimes, + "count of retry to create producer is " + createProducerCounter.get()); + }); + CompletableFuture topicCloseFuture = persistentTopic.close(true); + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { + String state = String.valueOf(replicator.getState()); + assertTrue(state.equals("Stopped") || state.equals("Terminated")); + }); + + // Delay close cursor, until "replicator.producer" create successfully. + // The next once retry time of create "replicator.producer" will be 3.2s. + Thread.sleep(4 * 1000); + log.info("Replicator.state: {}", replicator.getState()); + cursorCloseSignal.startClose(); + cursorCloseSignal.startCallback(); + + // Wait for topic close successfully. + // Verify there is no orphan producer on the remote cluster. + topicCloseFuture.join(); + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertEquals(persistentTopic2.getProducers().size(), 0); Assert.assertFalse(replicator.isConnected()); }); + // cleanup. cleanupTopics(() -> { admin1.topics().delete(topicName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 33620716288af..8e8b444f952c7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -18,21 +18,28 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.google.common.collect.Sets; import java.net.URL; +import java.time.Duration; import java.util.Collections; import java.util.Optional; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.testng.Assert; @Slf4j public abstract class OneWayReplicatorTestBase extends TestRetrySupport { @@ -140,10 +147,32 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { + waitChangeEventsInit(defaultNamespace); admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Collections.singleton(cluster1)); admin1.namespaces().unload(defaultNamespace); cleanupTopicAction.run(); admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); + waitChangeEventsInit(defaultNamespace); + } + + protected void waitChangeEventsInit(String namespace) { + PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) + .join().get(); + Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { + TopicStatsImpl topicStats = topic.getStats(true, false, false); + topicStats.getSubscriptions().entrySet().forEach(entry -> { + // No wait for compaction. + if (COMPACTION_SUBSCRIPTION.equals(entry.getKey())) { + return; + } + // No wait for durable cursor. + if (entry.getValue().isDurable()) { + return; + } + Assert.assertTrue(entry.getValue().getMsgBacklog() == 0, entry.getKey()); + }); + }); } protected interface CleanupTopicAction { @@ -166,7 +195,7 @@ protected void setup() throws Exception { log.info("--- OneWayReplicatorTestBase::setup completed ---"); } - private void setConfigDefaults(ServiceConfiguration config, String clusterName, + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { config.setClusterName(clusterName); config.setAdvertisedAddress("localhost"); @@ -185,10 +214,19 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); } @Override protected void cleanup() throws Exception { + // delete namespaces. + waitChangeEventsInit(defaultNamespace); + admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(defaultNamespace); + admin2.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster2)); + admin2.namespaces().deleteNamespace(defaultNamespace); + + // shutdown. markCurrentSetupNumberCleaned(); log.info("--- Shutting down ---"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index d5044276a5a63..de9d0272fc002 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -1799,12 +1799,12 @@ public void testClosingReplicationProducerTwice() throws Exception { any(), eq(null) ); - replicator.disconnect(false); - replicator.disconnect(false); + replicator.terminate(); + replicator.terminate(); replicator.startProducer(); - verify(clientImpl, Mockito.times(2)).createProducerAsync(any(), any(), any()); + verify(clientImpl, Mockito.times(1)).createProducerAsync(any(), any(), any()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 88a668e8745d5..a05c3468ea16e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -895,7 +895,7 @@ public void testReplicatorProducerClosing() throws Exception { pulsar2 = null; pulsar3.close(); pulsar3 = null; - replicator.disconnect(false); + replicator.terminate(); Thread.sleep(100); Field field = AbstractReplicator.class.getDeclaredField("producer"); field.setAccessible(true); @@ -1834,7 +1834,7 @@ public void testReplicatorWithTTL() throws Exception { persistentTopic.getReplicators().forEach((cluster, replicator) -> { PersistentReplicator persistentReplicator = (PersistentReplicator) replicator; // Pause replicator - persistentReplicator.disconnect(); + pauseReplicator(persistentReplicator); }); persistentProducer1.send("V2".getBytes()); @@ -1874,4 +1874,11 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + + private void pauseReplicator(PersistentReplicator replicator) { + Awaitility.await().untilAsserted(() -> { + assertTrue(replicator.isConnected()); + }); + replicator.closeProducerAsync(true); + } } From bfebe26cf461a08da12a2b4de05f101088674c53 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 23 Apr 2024 11:29:19 +0800 Subject: [PATCH 52/55] [improve][misc] Upgrade slf4j to 2.0.13 Signed-off-by: Zixuan Liu --- buildtools/pom.xml | 10 ++++++++-- distribution/server/pom.xml | 2 +- distribution/server/src/assemble/LICENSE.bin.txt | 6 +++--- distribution/shell/pom.xml | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 14 +++++++++++++- pulsar-client-all/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-io/alluxio/pom.xml | 10 ++++++++++ pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/solr/pom.xml | 4 ++++ structured-event-log/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 6 +++++- 17 files changed, 55 insertions(+), 19 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index cd4d02af3d7b4..58f99e9ea86b5 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -41,7 +41,7 @@ 1.8 3.1.0 2.23.1 - 1.7.32 + 2.0.13 7.7.1 3.11 4.1 @@ -100,6 +100,12 @@ org.testng testng ${testng.version} + + + org.slf4j + * + + org.apache.logging.log4j @@ -111,7 +117,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.slf4j diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 599a9755f9155..1c9ea68685308 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -180,7 +180,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c5642503b25b0..c5c243796b6f3 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -350,7 +350,7 @@ The Apache Software License, Version 2.0 * Log4J - org.apache.logging.log4j-log4j-api-2.23.1.jar - org.apache.logging.log4j-log4j-core-2.23.1.jar - - org.apache.logging.log4j-log4j-slf4j-impl-2.23.1.jar + - org.apache.logging.log4j-log4j-slf4j2-impl-2.23.1.jar - org.apache.logging.log4j-log4j-web-2.23.1.jar * Java Native Access JNA - net.java.dev.jna-jna-jpms-5.12.1.jar @@ -563,8 +563,8 @@ BSD 2-Clause License MIT License * Java SemVer -- com.github.zafarkhaja-java-semver-0.9.0.jar -- ../licenses/LICENSE-SemVer.txt * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - org.slf4j-slf4j-api-1.7.32.jar - - org.slf4j-jcl-over-slf4j-1.7.32.jar + - org.slf4j-slf4j-api-2.0.13.jar + - org.slf4j-jcl-over-slf4j-2.0.13.jar * The Checker Framework - org.checkerframework-checker-qual-3.33.0.jar * oshi diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 5f4fc549ccc62..144f7b1ff6d83 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 91d4643d9d4bc..f74e469ab20bf 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -385,7 +385,7 @@ The Apache Software License, Version 2.0 * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar - - log4j-slf4j-impl-2.23.1.jar + - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - opentelemetry-api-1.34.1.jar @@ -423,7 +423,7 @@ BSD 3-clause "New" or "Revised" License MIT License * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - slf4j-api-1.7.32.jar + - slf4j-api-2.0.13.jar * The Checker Framework - checker-qual-3.33.0.jar diff --git a/pom.xml b/pom.xml index 90b6c8cb8edf4..ad4ecd027c552 100644 --- a/pom.xml +++ b/pom.xml @@ -153,7 +153,7 @@ flexible messaging model and an intuitive client API. 0.16.0 4.3.8 7.9.2 - 1.7.32 + 2.0.13 4.4 2.23.1 1.78 @@ -352,6 +352,10 @@ flexible messaging model and an intuitive client API. org.yaml * + + org.slf4j + * + @@ -460,6 +464,10 @@ flexible messaging model and an intuitive client API. org.bouncycastle * + + log4j-slf4j-impl + org.apache.logging.log4j + slf4j-log4j12 org.slf4j @@ -1582,6 +1590,10 @@ flexible messaging model and an intuitive client API. org.apache.zookeeper * + + log4j-slf4j-impl + org.apache.logging.log4j + diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 781c03721fb69..eca03ee1502b0 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -66,7 +66,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index b8d197c0683d3..84d8a032d3bde 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -36,7 +36,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index 5118a30b92f94..001780bdb5178 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -43,7 +43,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 5e300c78aac46..d367038ff4945 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -105,7 +105,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 53f2ed095c683..86d76ec9578ee 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -57,6 +57,12 @@ org.alluxio alluxio-core-client-fs ${alluxio.version} + + + log4j-slf4j-impl + org.apache.logging.log4j + + @@ -69,6 +75,10 @@ org.glassfish javax.el + + log4j-slf4j-impl + org.apache.logging.log4j + diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index ef198579544a4..38f1460a33c57 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -61,7 +61,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index dcb0226a1f8d0..1a7a975098bec 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -65,7 +65,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 9074faed3b7cf..f09d2700f320e 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -49,7 +49,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 5be2639c718fb..2b7893fc945a1 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -70,6 +70,10 @@ jose4j org.bitbucket.b_c + + log4j-slf4j-impl + org.apache.logging.log4j + test diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index d09637fca76ae..cc34f921dc2bb 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 040d17a8b40da..8df8aa21c42f6 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -48,7 +48,7 @@ org.slf4j - slf4j-log4j12 + * @@ -118,6 +118,10 @@ org.bouncycastle * + + org.slf4j + * + From 0bfc087208bea8efa9092b2c10af8a23272c0883 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 23 Apr 2024 16:19:40 +0800 Subject: [PATCH 53/55] Fix rabbitmq test Signed-off-by: Zixuan Liu --- pulsar-io/rabbitmq/pom.xml | 2 +- .../io/rabbitmq/RabbitMQBrokerManager.java | 33 ++++++++++--------- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index f09d2700f320e..f8a5c0601d146 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -83,7 +83,7 @@ org.apache.qpid qpid-broker - 6.1.6 + 9.2.0 test diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java index 507313c86fd7f..83331bf7de810 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java @@ -18,28 +18,29 @@ */ package org.apache.pulsar.io.rabbitmq; -import org.apache.qpid.server.Broker; -import org.apache.qpid.server.BrokerOptions; - import java.io.File; import java.io.FileOutputStream; import java.nio.file.Files; import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import org.apache.qpid.server.SystemLauncher; +import org.apache.qpid.server.model.SystemConfig; public class RabbitMQBrokerManager { - private final Broker broker = new Broker(); + private final SystemLauncher systemLauncher = new SystemLauncher(); public void startBroker(String port) throws Exception { - BrokerOptions brokerOptions = getBrokerOptions(port); - broker.startup(brokerOptions); + Map brokerOptions = getBrokerOptions(port); + systemLauncher.startup(brokerOptions); } public void stopBroker() { - broker.shutdown(); + systemLauncher.shutdown(); } - BrokerOptions getBrokerOptions(String port) throws Exception { + Map getBrokerOptions(String port) throws Exception { Path tmpFolder = Files.createTempDirectory("qpidWork"); Path homeFolder = Files.createTempDirectory("qpidHome"); File etc = new File(homeFolder.toFile(), "etc"); @@ -48,15 +49,17 @@ BrokerOptions getBrokerOptions(String port) throws Exception { fos.write("guest:guest\n".getBytes()); fos.close(); - BrokerOptions brokerOptions = new BrokerOptions(); - - brokerOptions.setConfigProperty("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); - brokerOptions.setConfigProperty("qpid.amqp_port", port); - brokerOptions.setConfigProperty("qpid.home_dir", homeFolder.toAbsolutePath().toString()); + Map config = new HashMap<>(); + config.put("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); + config.put("qpid.amqp_port", port); + config.put("qpid.home_dir", homeFolder.toAbsolutePath().toString()); String configPath = getFile("qpid.json").getAbsolutePath(); - brokerOptions.setInitialConfigurationLocation(configPath); - return brokerOptions; + Map context = new HashMap<>(); + context.put(SystemConfig.INITIAL_CONFIGURATION_LOCATION, configPath); + context.put(SystemConfig.TYPE, "Memory"); + context.put(SystemConfig.CONTEXT, config); + return context; } private File getFile(String name) { From cbb2d63f20e556611ac5277f4558d8af239232d4 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 23 Apr 2024 23:46:51 +0800 Subject: [PATCH 54/55] Fix build Signed-off-by: Zixuan Liu --- pulsar-io/rabbitmq/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index f8a5c0601d146..a5fa705536bf5 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -85,6 +85,13 @@ qpid-broker 9.2.0 test + + + + org.apache.qpid + qpid-bdbstore + + org.awaitility From 962852ccd6df4eaac9767a4337061de279a63754 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 24 Apr 2024 02:13:56 +0800 Subject: [PATCH 55/55] Fix build Signed-off-by: Zixuan Liu --- pom.xml | 8 ++++++++ pulsar-io/rabbitmq/pom.xml | 7 ------- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index ad4ecd027c552..585347fb1f855 100644 --- a/pom.xml +++ b/pom.xml @@ -2713,5 +2713,13 @@ flexible messaging model and an intuitive client API. false + + + oracle.releases + https://download.oracle.com/maven + + false + + diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index a5fa705536bf5..f8a5c0601d146 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -85,13 +85,6 @@ qpid-broker 9.2.0 test - - - - org.apache.qpid - qpid-bdbstore - - org.awaitility