You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by pe...@apache.org on 2021/04/22 04:36:37 UTC

[pulsar] branch master updated: Support advertisedListeners for standalone (#10297)

This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new e9c336e  Support advertisedListeners for standalone (#10297)
e9c336e is described below

commit e9c336e66f5e989f0f58d5b35e01cb53d7c1c34e
Author: feynmanlin <fe...@tencent.com>
AuthorDate: Thu Apr 22 12:35:57 2021 +0800

    Support advertisedListeners for standalone (#10297)
    
    ### Motivation
    Now `advertisedListeners` and `advertisedAddress` can not appear together.
    In MultipleListenerValidator#validateAndAnalysisAdvertisedListener :
    ```
    if (StringUtils.isNotBlank(config.getAdvertisedListeners()) && StringUtils.isNotBlank(config.getAdvertisedAddress())) {
          throw new IllegalArgumentException("`advertisedListeners` and `advertisedAddress` must not appear together");
    }
    ```
    
    ### Modifications
    Only set the default localhost when both `AdvertisedAddress` and `advertisedListeners` are not set
    
    ### Verifying this change
---
 .../org/apache/pulsar/PulsarStandaloneStarter.java |  4 +-
 .../pulsar/broker/service/StandaloneTest.java      | 50 +++++++++++
 .../pulsar_broker_test_standalone.conf             | 96 ++++++++++++++++++++++
 3 files changed, 148 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java
index 62ce813..ac452b4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java
@@ -65,11 +65,11 @@ public class PulsarStandaloneStarter extends PulsarStandalone {
             // Use advertised address from command line
             config.setAdvertisedAddress(this.getAdvertisedAddress());
             zkServers = this.getAdvertisedAddress();
-        } else if (isBlank(config.getAdvertisedAddress())) {
+        } else if (isBlank(config.getAdvertisedAddress()) && isBlank(config.getAdvertisedListeners())) {
             // Use advertised address as local hostname
             config.setAdvertisedAddress("localhost");
         } else {
-            // Use advertised address from config file
+            // Use advertised or advertisedListeners address from config file
         }
 
         // Set ZK server's host to localhost
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java
new file mode 100644
index 0000000..8a4da94
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import org.apache.pulsar.PulsarStandaloneStarter;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.testng.annotations.Test;
+
+import static org.testng.AssertJUnit.assertEquals;
+import static org.testng.AssertJUnit.assertNull;
+
+@Test(groups = "broker")
+public class StandaloneTest extends MockedPulsarServiceBaseTest {
+
+    @Override
+    protected void setup() throws Exception {
+
+    }
+
+    @Override
+    protected void cleanup() throws Exception {
+
+    }
+
+    @Test
+    public void testAdvertised() throws Exception {
+        String args[] = new String[]{"--config",
+                "./src/test/resources/configurations/pulsar_broker_test_standalone.conf"};
+        PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(args);
+        assertNull(standalone.getConfig().getAdvertisedAddress());
+        assertEquals(standalone.getConfig().getAdvertisedListeners(),
+                "internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651");
+    }
+}
diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf
new file mode 100644
index 0000000..901193a
--- /dev/null
+++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf
@@ -0,0 +1,96 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+applicationName="pulsar_broker"
+zookeeperServers=
+configurationStoreServers="localhost"
+brokerServicePort=6650
+brokerServicePortTls=6651
+webServicePort=8080
+webServicePortTls=4443
+bindAddress=0.0.0.0
+advertisedAddress=
+advertisedListeners=internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651
+internalListenerName=internal
+clusterName="test_cluster"
+brokerShutdownTimeoutMs=3000
+backlogQuotaCheckEnabled=true
+backlogQuotaCheckIntervalInSeconds=60
+backlogQuotaDefaultLimitGB=50
+brokerDeleteInactiveTopicsEnabled=true
+brokerDeleteInactiveTopicsFrequencySeconds=60
+allowAutoTopicCreation=true
+allowAutoTopicCreationType=non-partitioned
+defaultNumPartitions=1
+messageExpiryCheckIntervalInMinutes=5
+clientLibraryVersionCheckEnabled=false
+clientLibraryVersionCheckAllowUnversioned=true
+statusFilePath=/tmp/status.html
+tlsEnabled=false
+tlsCertificateFilePath=/usr/local/conf/pulsar/server.crt
+tlsKeyFilePath=/home/local/conf/pulsar/server.key
+tlsTrustCertsFilePath=
+tlsAllowInsecureConnection=false
+authenticationEnabled=false
+authorizationEnabled=false
+superUserRoles="test_user"
+brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled"
+brokerClientAuthenticationParameters=
+bookkeeperClientAuthenticationPlugin="test_auth_plugin"
+bookkeeperClientAuthenticationAppId="test_auth_id"
+bookkeeperClientTimeoutInSeconds=30
+bookkeeperClientSpeculativeReadTimeoutInMillis=0
+bookkeeperClientHealthCheckEnabled=true
+bookkeeperClientHealthCheckIntervalSeconds=60
+bookkeeperClientHealthCheckErrorThresholdPerInterval=5
+bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800
+bookkeeperClientRackawarePolicyEnabled=true
+bookkeeperClientRegionawarePolicyEnabled=false
+bookkeeperClientMinNumRacksPerWriteQuorum=2
+bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false
+bookkeeperClientReorderReadSequenceEnabled=false
+bookkeeperClientIsolationGroups="test_group"
+managedLedgerDefaultEnsembleSize=3
+managedLedgerDefaultWriteQuorum=2
+managedLedgerDefaultAckQuorum=2
+managedLedgerCacheSizeMB=1024
+managedLedgerCacheEvictionWatermark=10
+managedLedgerDefaultMarkDeleteRateLimit=0.1
+managedLedgerMaxEntriesPerLedger=50000
+managedLedgerMinLedgerRolloverTimeMinutes=10
+managedLedgerMaxLedgerRolloverTimeMinutes=240
+managedLedgerCursorMaxEntriesPerLedger=50000
+managedLedgerCursorRolloverTimeInSeconds = 14400
+managedLedgerDataReadPriority = bookkeeper-first
+loadBalancerEnabled = false
+loadBalancerReportUpdateThresholdPercentage=10
+loadBalancerReportUpdateMaxIntervalMinutes=15
+loadBalancerHostUsageCheckIntervalMinutes=1
+loadBalancerSheddingIntervalMinutes=30
+loadBalancerSheddingGracePeriodMinutes=30
+loadBalancerBrokerUnderloadedThresholdPercentage=50
+loadBalancerBrokerOverloadedThresholdPercentage=85
+replicationMetricsEnabled=true
+replicationConnectionsPerBroker=16
+replicationProducerQueueSize=1000
+replicatorPrefix=pulsar.repl
+brokerDeleteInactiveTopicsMode=delete_when_subscriptions_caught_up
+supportedNamespaceBundleSplitAlgorithms=[range_equally_divide]
+defaultNamespaceBundleSplitAlgorithm=topic_count_equally_divide
+maxMessagePublishBufferSizeInMB=-1