You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by mm...@apache.org on 2018/02/26 04:57:43 UTC
[incubator-pulsar] branch master updated: Renamed DestinationName
into TopicName (#1280)
This is an automated email from the ASF dual-hosted git repository.
mmerli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pulsar.git
The following commit(s) were added to refs/heads/master by this push:
new 5fc4d53 Renamed DestinationName into TopicName (#1280)
5fc4d53 is described below
commit 5fc4d536d76b1d73c6be7b9238bf14b210ee095f
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Sun Feb 25 20:57:40 2018 -0800
Renamed DestinationName into TopicName (#1280)
---
.../mledger/impl/ManagedLedgerOfflineBacklog.java | 20 +-
.../authorization/AuthorizationProvider.java | 39 +-
.../broker/authorization/AuthorizationService.java | 154 ++++----
.../authorization/PulsarAuthorizationProvider.java | 102 ++---
.../org/apache/pulsar/broker/PulsarService.java | 12 +-
.../apache/pulsar/broker/admin/AdminResource.java | 32 +-
.../pulsar/broker/admin/impl/BrokerStatsBase.java | 8 +-
.../pulsar/broker/admin/impl/NamespacesBase.java | 16 +-
.../broker/admin/impl/PersistentTopicsBase.java | 440 ++++++++++-----------
.../pulsar/broker/admin/impl/PropertiesBase.java | 2 +-
.../apache/pulsar/broker/admin/v1/Namespaces.java | 31 +-
.../broker/admin/v1/NonPersistentTopics.java | 71 ++--
.../pulsar/broker/admin/v1/PersistentTopics.java | 172 ++++----
.../apache/pulsar/broker/admin/v2/Namespaces.java | 22 +-
.../broker/admin/v2/NonPersistentTopics.java | 68 ++--
.../pulsar/broker/admin/v2/PersistentTopics.java | 166 ++++----
.../loadbalance/impl/SimpleLoadCalculatorImpl.java | 2 +-
.../{DestinationLookup.java => TopicLookup.java} | 78 ++--
.../pulsar/broker/namespace/NamespaceService.java | 54 +--
.../pulsar/broker/service/AbstractReplicator.java | 22 +-
.../pulsar/broker/service/BacklogQuotaManager.java | 6 +-
.../pulsar/broker/service/BrokerService.java | 65 ++-
.../org/apache/pulsar/broker/service/Consumer.java | 11 +-
.../org/apache/pulsar/broker/service/Producer.java | 10 +-
.../apache/pulsar/broker/service/PulsarStats.java | 10 +-
.../apache/pulsar/broker/service/ServerCnx.java | 20 +-
.../apache/pulsar/broker/service/Subscription.java | 2 +-
.../nonpersistent/NonPersistentSubscription.java | 6 +-
.../service/nonpersistent/NonPersistentTopic.java | 86 ++--
.../service/persistent/CompactorSubscription.java | 2 +-
.../service/persistent/DispatchRateLimiter.java | 6 +-
.../service/persistent/MessageDeduplication.java | 4 +-
.../service/persistent/PersistentSubscription.java | 6 +-
.../broker/service/persistent/PersistentTopic.java | 136 +++----
.../broker/stats/BookieClientStatsGenerator.java | 16 +-
.../pulsar/broker/stats/MetricsGenerator.java | 2 +-
.../broker/stats/metrics/AbstractMetrics.java | 14 +-
.../broker/stats/metrics/ManagedLedgerMetrics.java | 16 +-
.../pulsar/broker/web/PulsarWebResource.java | 34 +-
.../pulsar/common/naming/NamespaceBundle.java | 8 +-
.../pulsar/common/naming/NamespaceBundles.java | 10 +-
.../apache/pulsar/broker/SLAMonitoringTest.java | 20 +-
.../apache/pulsar/broker/admin/AdminApiTest.java | 58 +--
.../apache/pulsar/broker/admin/AdminApiTest2.java | 16 +-
.../org/apache/pulsar/broker/admin/AdminTest.java | 22 +-
.../broker/admin/CreateSubscriptionTest.java | 4 +-
.../broker/admin/IncrementPartitionsTest.java | 4 +-
.../apache/pulsar/broker/admin/NamespacesTest.java | 20 +-
.../pulsar/broker/auth/AuthorizationTest.java | 158 ++++----
.../broker/loadbalance/LoadBalancerTest.java | 24 +-
...ookupv2Test.java => HttpTopicLookupv2Test.java} | 20 +-
.../broker/namespace/NamespaceServiceTest.java | 30 +-
.../broker/service/BrokerBkEnsemblesTests.java | 30 +-
.../pulsar/broker/service/BrokerServiceTest.java | 22 +-
.../PersistentDispatcherFailoverConsumerTest.java | 4 +-
.../broker/service/PersistentFailoverE2ETest.java | 4 +-
.../service/PersistentTopicConcurrentTest.java | 4 +-
.../broker/service/PersistentTopicE2ETest.java | 18 +-
.../pulsar/broker/service/PersistentTopicTest.java | 6 +-
.../pulsar/broker/service/ReplicatorTest.java | 36 +-
.../pulsar/broker/service/ReplicatorTestBase.java | 18 +-
.../pulsar/broker/service/ServerCnxTest.java | 24 +-
.../zookeeper/ZooKeeperClientAspectJTest.java | 4 +-
.../api/AuthenticatedProducerConsumerTest.java | 6 +-
.../api/AuthorizationProducerConsumerTest.java | 50 +--
.../pulsar/client/api/BrokerServiceLookupTest.java | 30 +-
.../pulsar/client/api/NonPersistentTopicTest.java | 18 +-
.../api/PartitionedProducerConsumerTest.java | 97 +++--
.../client/api/SimpleProducerConsumerTest.java | 8 +-
.../client/impl/BrokerClientIntegrationTest.java | 28 +-
.../pulsar/common/naming/NamespaceBundleTest.java | 14 +-
.../pulsar/common/naming/NamespaceBundlesTest.java | 26 +-
.../websocket/proxy/ProxyAuthorizationTest.java | 26 +-
.../apache/pulsar/client/admin/BrokerStats.java | 12 +-
.../org/apache/pulsar/client/admin/Lookup.java | 21 +-
.../org/apache/pulsar/client/admin/Namespaces.java | 34 +-
.../pulsar/client/admin/NonPersistentTopics.java | 72 ++--
.../pulsar/client/admin/PersistentTopics.java | 294 +++++++-------
.../org/apache/pulsar/client/admin/Properties.java | 2 +-
.../apache/pulsar/client/admin/PulsarAdmin.java | 6 +-
.../client/admin/internal/BrokerStatsImpl.java | 2 +-
.../pulsar/client/admin/internal/LookupImpl.java | 14 +-
.../client/admin/internal/NamespacesImpl.java | 2 +-
.../admin/internal/NonPersistentTopicsImpl.java | 48 +--
.../admin/internal/PersistentTopicsImpl.java | 194 ++++-----
.../clients/consumer/PulsarKafkaConsumer.java | 10 +-
.../org/apache/pulsar/admin/cli/CliCommand.java | 26 +-
.../apache/pulsar/admin/cli/CmdBrokerStats.java | 10 +-
.../org/apache/pulsar/admin/cli/CmdNamespaces.java | 11 +-
.../pulsar/admin/cli/CmdNonPersistentTopics.java | 20 +-
.../pulsar/admin/cli/CmdPersistentTopics.java | 84 ++--
.../pulsar/admin/cli/PulsarAdminToolTest.java | 10 +-
.../client/impl/BinaryProtoLookupService.java | 30 +-
.../pulsar/client/impl/HttpLookupService.java | 14 +-
.../apache/pulsar/client/impl/LookupService.java | 10 +-
.../client/impl/PartitionedConsumerImpl.java | 4 +-
.../client/impl/PartitionedProducerImpl.java | 4 +-
.../pulsar/client/impl/PulsarClientImpl.java | 20 +-
.../pulsar/client/impl/TopicsConsumerImpl.java | 28 +-
.../apache/pulsar/common/naming/NamedEntity.java | 2 +-
.../apache/pulsar/common/naming/NamespaceName.java | 18 +-
.../apache/pulsar/common/naming/ServiceUnitId.java | 8 +-
.../{DestinationDomain.java => TopicDomain.java} | 8 +-
.../{DestinationName.java => TopicName.java} | 104 ++---
.../pulsar/common/naming/NamespaceNameTest.java | 10 +-
...DestinationNameTest.java => TopicNameTest.java} | 130 +++---
.../discovery/service/BrokerDiscoveryProvider.java | 22 +-
.../pulsar/discovery/service/ServerConnection.java | 10 +-
.../discovery/service/DiscoveryServiceTest.java | 6 +-
.../proxy/server/BrokerDiscoveryProvider.java | 22 +-
.../pulsar/proxy/server/LookupProxyHandler.java | 18 +-
.../proxy/socket/client/PerformanceClient.java | 10 +-
.../pulsar/testclient/LoadSimulationClient.java | 8 +-
.../testclient/LoadSimulationController.java | 70 ++--
.../pulsar/testclient/PerformanceConsumer.java | 24 +-
.../pulsar/testclient/PerformanceProducer.java | 8 +-
.../pulsar/testclient/PerformanceReader.java | 16 +-
.../pulsar/websocket/AbstractWebSocketHandler.java | 6 +-
.../apache/pulsar/websocket/ConsumerHandler.java | 4 +-
.../apache/pulsar/websocket/ProducerHandler.java | 4 +-
.../org/apache/pulsar/websocket/ReaderHandler.java | 4 +-
.../websocket/admin/WebSocketProxyStats.java | 18 +-
.../websocket/admin/WebSocketWebResource.java | 6 +-
.../apache/pulsar/websocket/stats/ProxyStats.java | 6 +-
124 files changed, 2172 insertions(+), 2182 deletions(-)
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
index ee22c5e..c07ed55 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java
@@ -36,7 +36,7 @@ import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.proto.MLDataFormats;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.slf4j.Logger;
@@ -100,12 +100,12 @@ public class ManagedLedgerOfflineBacklog {
public PersistentOfflineTopicStats getEstimatedUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory,
String managedLedgerName) throws Exception {
- return estimateUnloadedTopicBacklog(factory, DestinationName.get("persistent://" + managedLedgerName));
+ return estimateUnloadedTopicBacklog(factory, TopicName.get("persistent://" + managedLedgerName));
}
public PersistentOfflineTopicStats estimateUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory,
- DestinationName dn) throws Exception {
- String managedLedgerName = dn.getPersistenceNamingEncoding();
+ TopicName topicName) throws Exception {
+ String managedLedgerName = topicName.getPersistenceNamingEncoding();
long numberOfEntries = 0;
long totalSize = 0;
final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers = new ConcurrentSkipListMap<>();
@@ -113,7 +113,7 @@ public class ManagedLedgerOfflineBacklog {
brokerName);
// calculate total managed ledger size and number of entries without loading the topic
- readLedgerMeta(factory, dn, ledgers);
+ readLedgerMeta(factory, topicName, ledgers);
for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers.values()) {
numberOfEntries += ls.getEntries();
totalSize += ls.getSize();
@@ -128,15 +128,15 @@ public class ManagedLedgerOfflineBacklog {
}
// calculate per cursor message backlog
- calculateCursorBacklogs(factory, dn, ledgers, offlineTopicStats);
+ calculateCursorBacklogs(factory, topicName, ledgers, offlineTopicStats);
offlineTopicStats.statGeneratedAt.setTime(System.currentTimeMillis());
return offlineTopicStats;
}
- private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final DestinationName dn,
+ private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final TopicName topicName,
final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers) throws Exception {
- String managedLedgerName = dn.getPersistenceNamingEncoding();
+ String managedLedgerName = topicName.getPersistenceNamingEncoding();
MetaStore store = factory.getMetaStore();
BookKeeper bk = factory.getBookKeeper();
final CountDownLatch mlMetaCounter = new CountDownLatch(1);
@@ -206,14 +206,14 @@ public class ManagedLedgerOfflineBacklog {
}
}
- private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, final DestinationName dn,
+ private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, final TopicName topicName,
final NavigableMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgers,
final PersistentOfflineTopicStats offlineTopicStats) throws Exception {
if (ledgers.size() == 0) {
return;
}
- String managedLedgerName = dn.getPersistenceNamingEncoding();
+ String managedLedgerName = topicName.getPersistenceNamingEncoding();
MetaStore store = factory.getMetaStore();
BookKeeper bk = factory.getBookKeeper();
final CountDownLatch allCursorsCounter = new CountDownLatch(1);
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
index 9962c05..cae2415 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java
@@ -26,7 +26,7 @@ import java.util.concurrent.CompletableFuture;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.AuthAction;
@@ -48,47 +48,46 @@ public interface AuthorizationProvider extends Closeable {
void initialize(ServiceConfiguration conf, ConfigurationCacheService configCache) throws IOException;
/**
- * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+ * Check if the specified role has permission to send messages to the specified fully qualified topic name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to send messages to the destination.
+ * the app id used to send messages to the topic.
*/
- CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData);
/**
- * Check if the specified role has permission to receive messages from the specified fully qualified destination
- * name.
+ * Check if the specified role has permission to receive messages from the specified fully qualified topic name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to receive messages from the destination.
+ * the app id used to receive messages from the topic.
* @param subscription
* the subscription name defined by the client
*/
- CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription);
/**
- * Check whether the specified role can perform a lookup for the specified destination.
+ * Check whether the specified role can perform a lookup for the specified topic.
*
* For that the caller needs to have producer or consumer permission.
*
- * @param destination
+ * @param topicName
* @param role
* @return
* @throws Exception
*/
- CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData);
/**
- *
+ *
* Grant authorization-action permission on a namespace to the given client
- *
+ *
* @param namespace
* @param actions
* @param role
@@ -104,8 +103,8 @@ public interface AuthorizationProvider extends Closeable {
/**
* Grant authorization-action permission on a topic to the given client
- *
- * @param topicname
+ *
+ * @param topicName
* @param role
* @param authDataJson
* additional authdata in json format
@@ -114,7 +113,7 @@ public interface AuthorizationProvider extends Closeable {
* IllegalArgumentException when namespace not found<br/>
* IllegalStateException when failed to grant permission
*/
- CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions, String role,
+ CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<AuthAction> actions, String role,
String authDataJson);
}
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
index 8cb9232..f38482d 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java
@@ -34,7 +34,7 @@ import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.util.FutureUtil;
@@ -45,7 +45,7 @@ import com.google.common.collect.Maps;
/**
* Authorization service that manages pluggable authorization provider and authorize requests accordingly.
- *
+ *
*/
public class AuthorizationService {
private static final Logger log = LoggerFactory.getLogger(AuthorizationService.class);
@@ -60,16 +60,16 @@ public class AuthorizationService {
if (this.conf.isAuthorizationEnabled()) {
try {
final String providerClassname = conf.getAuthorizationProvider();
- if(StringUtils.isNotBlank(providerClassname)) {
+ if (StringUtils.isNotBlank(providerClassname)) {
provider = (AuthorizationProvider) Class.forName(providerClassname).newInstance();
provider.initialize(conf, configCache);
- log.info("{} has been loaded.", providerClassname);
+ log.info("{} has been loaded.", providerClassname);
} else {
throw new PulsarServerException("No authorization providers are present.");
}
} catch (PulsarServerException e) {
throw e;
- }catch (Throwable e) {
+ } catch (Throwable e) {
throw new PulsarServerException("Failed to load an authorization provider.", e);
}
} else {
@@ -78,9 +78,9 @@ public class AuthorizationService {
}
/**
- *
+ *
* Grant authorization-action permission on a namespace to the given client
- *
+ *
* @param namespace
* @param actions
* @param role
@@ -103,7 +103,7 @@ public class AuthorizationService {
/**
* Grant authorization-action permission on a topic to the given client
- *
+ *
* @param topicname
* @param role
* @param authDataJson
@@ -112,26 +112,25 @@ public class AuthorizationService {
* @throws IllegalStateException
* when failed to grant permission
*/
- public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions, String role,
+ public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions, String role,
String authDataJson) {
if (provider != null) {
return provider.grantPermissionAsync(topicname, actions, role, authDataJson);
}
- return FutureUtil
- .failedFuture(new IllegalStateException("No authorization provider configured"));
+ return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
}
/**
- * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+ * Check if the specified role has permission to send messages to the specified fully qualified topic name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to send messages to the destination.
+ * the app id used to send messages to the topic.
*/
- public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
if (!this.conf.isAuthorizationEnabled()) {
@@ -139,126 +138,121 @@ public class AuthorizationService {
}
if (provider != null) {
- return provider.canProduceAsync(destination, role, authenticationData);
+ return provider.canProduceAsync(topicName, role, authenticationData);
}
- return FutureUtil
- .failedFuture(new IllegalStateException("No authorization provider configured"));
+ return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
}
/**
- * Check if the specified role has permission to receive messages from the specified fully qualified destination
- * name.
+ * Check if the specified role has permission to receive messages from the specified fully qualified topic name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to receive messages from the destination.
+ * the app id used to receive messages from the topic.
* @param subscription
* the subscription name defined by the client
*/
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
if (!this.conf.isAuthorizationEnabled()) {
return CompletableFuture.completedFuture(true);
}
if (provider != null) {
- return provider.canConsumeAsync(destination, role, authenticationData, subscription);
+ return provider.canConsumeAsync(topicName, role, authenticationData, subscription);
}
- return FutureUtil
- .failedFuture(new IllegalStateException("No authorization provider configured"));
+ return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured"));
}
- public boolean canProduce(DestinationName destination, String role, AuthenticationDataSource authenticationData) throws Exception {
+ public boolean canProduce(TopicName topicName, String role, AuthenticationDataSource authenticationData)
+ throws Exception {
try {
- return canProduceAsync(destination, role, authenticationData).get(cacheTimeOutInSec,
- SECONDS);
+ return canProduceAsync(topicName, role, authenticationData).get(cacheTimeOutInSec, SECONDS);
} catch (InterruptedException e) {
- log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, destination);
+ log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, topicName);
throw e;
} catch (Exception e) {
- log.warn("Producer-client with Role - {} failed to get permissions for destination - {}. {}", role,
- destination, e.getMessage());
+ log.warn("Producer-client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
+ e.getMessage());
throw e;
}
}
- public boolean canConsume(DestinationName destination, String role, AuthenticationDataSource authenticationData,
+ public boolean canConsume(TopicName topicName, String role, AuthenticationDataSource authenticationData,
String subscription) throws Exception {
try {
- return canConsumeAsync(destination, role, authenticationData, subscription)
- .get(cacheTimeOutInSec, SECONDS);
+ return canConsumeAsync(topicName, role, authenticationData, subscription).get(cacheTimeOutInSec, SECONDS);
} catch (InterruptedException e) {
- log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, destination);
+ log.warn("Time-out {} sec while checking authorization on {} ", cacheTimeOutInSec, topicName);
throw e;
} catch (Exception e) {
- log.warn("Consumer-client with Role - {} failed to get permissions for destination - {}. {}", role,
- destination, e.getMessage());
+ log.warn("Consumer-client with Role - {} failed to get permissions for topic - {}. {}", role,
+ topicName, e.getMessage());
throw e;
}
}
/**
- * Check whether the specified role can perform a lookup for the specified destination.
+ * Check whether the specified role can perform a lookup for the specified topic.
*
* For that the caller needs to have producer or consumer permission.
*
- * @param destination
+ * @param topicName
* @param role
* @return
* @throws Exception
*/
- public boolean canLookup(DestinationName destination, String role, AuthenticationDataSource authenticationData) throws Exception {
- return canProduce(destination, role, authenticationData)
- || canConsume(destination, role, authenticationData, null);
+ public boolean canLookup(TopicName topicName, String role, AuthenticationDataSource authenticationData)
+ throws Exception {
+ return canProduce(topicName, role, authenticationData)
+ || canConsume(topicName, role, authenticationData, null);
}
/**
- * Check whether the specified role can perform a lookup for the specified destination.
+ * Check whether the specified role can perform a lookup for the specified topic.
*
* For that the caller needs to have producer or consumer permission.
*
- * @param destination
+ * @param topicName
* @param role
* @return
* @throws Exception
*/
- public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
CompletableFuture<Boolean> finalResult = new CompletableFuture<Boolean>();
- canProduceAsync(destination, role, authenticationData)
- .whenComplete((produceAuthorized, ex) -> {
- if (ex == null) {
- if (produceAuthorized) {
- finalResult.complete(produceAuthorized);
- return;
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug(
- "Destination [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
- destination.toString(), role, ex.getMessage());
- }
+ canProduceAsync(topicName, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
+ if (ex == null) {
+ if (produceAuthorized) {
+ finalResult.complete(produceAuthorized);
+ return;
+ }
+ } else {
+ if (log.isDebugEnabled()) {
+ log.debug(
+ "Topic [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
+ topicName.toString(), role, ex.getMessage());
+ }
+ }
+ canConsumeAsync(topicName, role, null, null).whenComplete((consumeAuthorized, e) -> {
+ if (e == null) {
+ if (consumeAuthorized) {
+ finalResult.complete(consumeAuthorized);
+ return;
}
- canConsumeAsync(destination, role, null, null)
- .whenComplete((consumeAuthorized, e) -> {
- if (e == null) {
- if (consumeAuthorized) {
- finalResult.complete(consumeAuthorized);
- return;
- }
- } else {
- if (log.isDebugEnabled()) {
- log.debug(
- "Destination [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
- destination.toString(), role, e.getMessage());
+ } else {
+ if (log.isDebugEnabled()) {
+ log.debug(
+ "Topic [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
+ topicName.toString(), role, e.getMessage());
- }
- finalResult.completeExceptionally(e);
- return;
- }
- finalResult.complete(false);
- });
- });
+ }
+ finalResult.completeExceptionally(e);
+ return;
+ }
+ finalResult.complete(false);
+ });
+ });
return finalResult;
}
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
index 7bd9246..a52d5c9 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java
@@ -31,7 +31,7 @@ import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.Policies;
@@ -74,39 +74,39 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
}
/**
- * Check if the specified role has permission to send messages to the specified fully qualified destination name.
+ * Check if the specified role has permission to send messages to the specified fully qualified topic name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to send messages to the destination.
+ * the app id used to send messages to the topic.
*/
@Override
- public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
- return checkAuthorization(destination, role, AuthAction.produce);
+ return checkAuthorization(topicName, role, AuthAction.produce);
}
/**
- * Check if the specified role has permission to receive messages from the specified fully qualified destination
+ * Check if the specified role has permission to receive messages from the specified fully qualified topic
* name.
*
- * @param destination
- * the fully qualified destination name associated with the destination.
+ * @param topicName
+ * the fully qualified topic name associated with the topic.
* @param role
- * the app id used to receive messages from the destination.
+ * the app id used to receive messages from the topic.
* @param subscription
* the subscription name defined by the client
*/
@Override
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
CompletableFuture<Boolean> permissionFuture = new CompletableFuture<>();
try {
- configCache.policiesCache().getAsync(POLICY_ROOT + destination.getNamespace()).thenAccept(policies -> {
+ configCache.policiesCache().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> {
if (!policies.isPresent()) {
if (log.isDebugEnabled()) {
- log.debug("Policies node couldn't be found for destination : {}", destination);
+ log.debug("Policies node couldn't be found for topic : {}", topicName);
}
} else {
if (isNotBlank(subscription) && !isSuperUser(role)) {
@@ -114,8 +114,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
case Prefix:
if (!subscription.startsWith(role)) {
PulsarServerException ex = new PulsarServerException(String.format(
- "Failed to create consumer - The subscription name needs to be prefixed by the authentication role, like %s-xxxx for destination: %s",
- role, destination));
+ "Failed to create consumer - The subscription name needs to be prefixed by the authentication role, like %s-xxxx for topic: %s",
+ role, topicName));
permissionFuture.completeExceptionally(ex);
return;
}
@@ -125,17 +125,17 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
}
}
}
- checkAuthorization(destination, role, AuthAction.consume).thenAccept(isAuthorized -> {
+ checkAuthorization(topicName, role, AuthAction.consume).thenAccept(isAuthorized -> {
permissionFuture.complete(isAuthorized);
});
}).exceptionally(ex -> {
- log.warn("Client with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+ log.warn("Client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
ex.getMessage());
permissionFuture.completeExceptionally(ex);
return null;
});
} catch (Exception e) {
- log.warn("Client with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+ log.warn("Client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
e.getMessage());
permissionFuture.completeExceptionally(e);
}
@@ -143,20 +143,20 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
}
/**
- * Check whether the specified role can perform a lookup for the specified destination.
+ * Check whether the specified role can perform a lookup for the specified topic.
*
* For that the caller needs to have producer or consumer permission.
*
- * @param destination
+ * @param topicName
* @param role
* @return
* @throws Exception
*/
@Override
- public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
CompletableFuture<Boolean> finalResult = new CompletableFuture<Boolean>();
- canProduceAsync(destination, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
+ canProduceAsync(topicName, role, authenticationData).whenComplete((produceAuthorized, ex) -> {
if (ex == null) {
if (produceAuthorized) {
finalResult.complete(produceAuthorized);
@@ -165,11 +165,11 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
} else {
if (log.isDebugEnabled()) {
log.debug(
- "Destination [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
- destination.toString(), role, ex.getMessage());
+ "Topic [{}] Role [{}] exception occured while trying to check Produce permissions. {}",
+ topicName.toString(), role, ex.getMessage());
}
}
- canConsumeAsync(destination, role, authenticationData, null).whenComplete((consumeAuthorized, e) -> {
+ canConsumeAsync(topicName, role, authenticationData, null).whenComplete((consumeAuthorized, e) -> {
if (e == null) {
if (consumeAuthorized) {
finalResult.complete(consumeAuthorized);
@@ -178,8 +178,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
} else {
if (log.isDebugEnabled()) {
log.debug(
- "Destination [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
- destination.toString(), role, e.getMessage());
+ "Topic [{}] Role [{}] exception occured while trying to check Consume permissions. {}",
+ topicName.toString(), role, e.getMessage());
}
finalResult.completeExceptionally(e);
@@ -192,9 +192,9 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
}
@Override
- public CompletableFuture<Void> grantPermissionAsync(DestinationName destination, Set<AuthAction> actions,
+ public CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<AuthAction> actions,
String role, String authDataJson) {
- return grantPermissionAsync(destination.getNamespaceObject(), actions, role, authDataJson);
+ return grantPermissionAsync(topicName.getNamespaceObject(), actions, role, authDataJson);
}
@Override
@@ -246,34 +246,34 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
return result;
}
- private CompletableFuture<Boolean> checkAuthorization(DestinationName destination, String role, AuthAction action) {
+ private CompletableFuture<Boolean> checkAuthorization(TopicName topicName, String role, AuthAction action) {
if (isSuperUser(role)) {
return CompletableFuture.completedFuture(true);
} else {
- return checkPermission(destination, role, action)
- .thenApply(isPermission -> isPermission && checkCluster(destination));
+ return checkPermission(topicName, role, action)
+ .thenApply(isPermission -> isPermission && checkCluster(topicName));
}
}
- private boolean checkCluster(DestinationName destination) {
- if (destination.isGlobal() || conf.getClusterName().equals(destination.getCluster())) {
+ private boolean checkCluster(TopicName topicName) {
+ if (topicName.isGlobal() || conf.getClusterName().equals(topicName.getCluster())) {
return true;
} else {
if (log.isDebugEnabled()) {
- log.debug("Destination [{}] does not belong to local cluster [{}]", destination.toString(),
+ log.debug("Topic [{}] does not belong to local cluster [{}]", topicName.toString(),
conf.getClusterName());
}
return false;
}
}
- public CompletableFuture<Boolean> checkPermission(DestinationName destination, String role, AuthAction action) {
+ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String role, AuthAction action) {
CompletableFuture<Boolean> permissionFuture = new CompletableFuture<>();
try {
- configCache.policiesCache().getAsync(POLICY_ROOT + destination.getNamespace()).thenAccept(policies -> {
+ configCache.policiesCache().getAsync(POLICY_ROOT + topicName.getNamespace()).thenAccept(policies -> {
if (!policies.isPresent()) {
if (log.isDebugEnabled()) {
- log.debug("Policies node couldn't be found for destination : {}", destination);
+ log.debug("Policies node couldn't be found for topic : {}", topicName);
}
} else {
Map<String, Set<AuthAction>> namespaceRoles = policies.get().auth_policies.namespace_auth;
@@ -284,13 +284,13 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
return;
}
- Map<String, Set<AuthAction>> destinationRoles = policies.get().auth_policies.destination_auth
- .get(destination.toString());
- if (destinationRoles != null) {
- // Destination has custom policy
- Set<AuthAction> destinationActions = destinationRoles.get(role);
- if (destinationActions != null && destinationActions.contains(action)) {
- // The role has destination level permission
+ Map<String, Set<AuthAction>> topicRoles = policies.get().auth_policies.destination_auth
+ .get(topicName.toString());
+ if (topicRoles != null) {
+ // Topic has custom policy
+ Set<AuthAction> topicActions = topicRoles.get(role);
+ if (topicActions != null && topicActions.contains(action)) {
+ // The role has topic level permission
permissionFuture.complete(true);
return;
}
@@ -304,8 +304,8 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
return;
}
- if (destinationRoles != null && checkWildcardPermission(role, action, destinationRoles)) {
- // The role has destination level permission by wildcard match
+ if (topicRoles != null && checkWildcardPermission(role, action, topicRoles)) {
+ // The role has topic level permission by wildcard match
permissionFuture.complete(true);
return;
}
@@ -313,13 +313,13 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
}
permissionFuture.complete(false);
}).exceptionally(ex -> {
- log.warn("Client with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+ log.warn("Client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
ex.getMessage());
permissionFuture.completeExceptionally(ex);
return null;
});
} catch (Exception e) {
- log.warn("Client with Role - {} failed to get permissions for destination - {}. {}", role, destination,
+ log.warn("Client with Role - {} failed to get permissions for topic - {}. {}", role, topicName,
e.getMessage());
permissionFuture.completeExceptionally(e);
}
@@ -352,7 +352,7 @@ public class PulsarAuthorizationProvider implements AuthorizationProvider {
* Super user roles are allowed to do anything, used for replication primarily
*
* @param role
- * the app id used to receive messages from the destination.
+ * the app id used to receive messages from the topic.
*/
public boolean isSuperUser(String role) {
Set<String> superUserRoles = conf.getSuperUserRoles();
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 bec354b..5ee1532 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
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.web.WebService;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
import org.apache.pulsar.common.configuration.VipStatus;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.ClusterData;
@@ -459,13 +459,13 @@ public class PulsarService implements AutoCloseable {
}
/**
- * Load all the destination contained in a namespace
+ * Load all the topics contained in a namespace
*
* @param bundle
* <code>NamespaceBundle</code> to identify the service unit
* @throws Exception
*/
- public void loadNamespaceDestinations(NamespaceBundle bundle) {
+ public void loadNamespaceTopics(NamespaceBundle bundle) {
executor.submit(() -> {
LOG.info("Loading all topics on bundle: {}", bundle);
@@ -473,10 +473,10 @@ public class PulsarService implements AutoCloseable {
List<CompletableFuture<Topic>> persistentTopics = Lists.newArrayList();
long topicLoadStart = System.nanoTime();
- for (String topic : getNamespaceService().getListOfDestinations(nsName)) {
+ for (String topic : getNamespaceService().getListOfTopics(nsName)) {
try {
- DestinationName dn = DestinationName.get(topic);
- if (bundle.includes(dn)) {
+ TopicName topicName = TopicName.get(topic);
+ if (bundle.includes(topicName)) {
CompletableFuture<Topic> future = brokerService.getTopic(topic);
if (future != null) {
persistentTopics.add(future);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
index b91c63a..527f455 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
import org.apache.pulsar.broker.web.PulsarWebResource;
import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -103,7 +103,7 @@ public abstract class AdminResource extends PulsarWebResource {
}
/**
- * Get the domain of the destination (whether it's queue or topic)
+ * Get the domain of the topic (whether it's persistent or non-persistent)
*/
protected String domain() {
if (uri.getPath().startsWith("persistent/")) {
@@ -237,28 +237,28 @@ public abstract class AdminResource extends PulsarWebResource {
}
}
- protected DestinationName destinationName;
+ protected TopicName topicName;
- protected void validateDestinationName(String property, String namespace, String encodedTopic) {
+ protected void validateTopicName(String property, String namespace, String encodedTopic) {
String topic = Codec.decode(encodedTopic);
try {
this.namespaceName = NamespaceName.get(property, namespace);
- this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+ this.topicName = TopicName.get(domain(), namespaceName, topic);
} catch (IllegalArgumentException e) {
log.warn("[{}] Failed to validate topic name {}://{}/{}/{}", clientAppId(), domain(), property, namespace,
topic, e);
throw new RestException(Status.PRECONDITION_FAILED, "Topic name is not valid");
}
- this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+ this.topicName = TopicName.get(domain(), namespaceName, topic);
}
@Deprecated
- protected void validateDestinationName(String property, String cluster, String namespace, String encodedTopic) {
+ protected void validateTopicName(String property, String cluster, String namespace, String encodedTopic) {
String topic = Codec.decode(encodedTopic);
try {
this.namespaceName = NamespaceName.get(property, cluster, namespace);
- this.destinationName = DestinationName.get(domain(), namespaceName, topic);
+ this.topicName = TopicName.get(domain(), namespaceName, topic);
} catch (IllegalArgumentException e) {
log.warn("[{}] Failed to validate topic name {}://{}/{}/{}/{}", clientAppId(), domain(), property, cluster,
namespace, topic, e);
@@ -358,30 +358,30 @@ public abstract class AdminResource extends PulsarWebResource {
return pulsar().getConfigurationCache().failureDomainListCache();
}
- protected PartitionedTopicMetadata getPartitionedTopicMetadata(DestinationName destinationName,
+ protected PartitionedTopicMetadata getPartitionedTopicMetadata(TopicName topicName,
boolean authoritative) {
- validateClusterOwnership(destinationName.getCluster());
+ validateClusterOwnership(topicName.getCluster());
// validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
// serve/redirect request else fail partitioned-metadata-request so, client fails while creating
// producer/consumer
- validateGlobalNamespaceOwnership(destinationName.getNamespaceObject());
+ validateGlobalNamespaceOwnership(topicName.getNamespaceObject());
try {
- checkConnect(destinationName);
+ checkConnect(topicName);
} catch (WebApplicationException e) {
- validateAdminAccessOnProperty(destinationName.getProperty());
+ validateAdminAccessOnProperty(topicName.getProperty());
} catch (Exception e) {
// unknown error marked as internal server error
- log.warn("Unexpected error while authorizing lookup. destination={}, role={}. Error: {}", destinationName,
+ log.warn("Unexpected error while authorizing lookup. topic={}, role={}. Error: {}", topicName,
clientAppId(), e.getMessage(), e);
throw new RestException(e);
}
- String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), destinationName.getEncodedLocalName());
+ String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), topicName.getEncodedLocalName());
PartitionedTopicMetadata partitionMetadata = fetchPartitionedTopicMetadata(pulsar(), path);
if (log.isDebugEnabled()) {
- log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), destinationName,
+ log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId(), topicName,
partitionMetadata.partitions);
}
return partitionMetadata;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
index 549a033..9da3fc6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java
@@ -87,12 +87,12 @@ public class BrokerStatsBase extends AdminResource {
@GET
@Path("/destinations")
- @ApiOperation(value = "Get all the destination stats by namesapce", response = OutputStream.class, responseContainer = "OutputStream") // https://github.com/swagger-api/swagger-ui/issues/558
+ @ApiOperation(value = "Get all the topic stats by namesapce", response = OutputStream.class, responseContainer = "OutputStream") // https://github.com/swagger-api/swagger-ui/issues/558
// map
// support
// missing
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
- public StreamingOutput getDestinations2() throws Exception {
+ public StreamingOutput getTopics2() throws Exception {
// Ensure super user access only
validateSuperUserAccess();
return output -> pulsar().getBrokerService().getDimensionMetrics(statsBuf -> {
@@ -138,14 +138,14 @@ public class BrokerStatsBase extends AdminResource {
try {
return BookieClientStatsGenerator.generate(pulsar());
} catch (Exception e) {
- log.error("[{}] Failed to generate pending bookie ops stats for destinations", clientAppId(), e);
+ log.error("[{}] Failed to generate pending bookie ops stats for topicss", clientAppId(), e);
throw new RestException(e);
}
}
@GET
@Path("/load-report")
- @ApiOperation(value = "Get Load for this broker", notes = "consists of destinationstats & systemResourceUsage", response = LoadReport.class)
+ @ApiOperation(value = "Get Load for this broker", notes = "consists of topics stats & systemResourceUsage", response = LoadReport.class)
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
public LoadManagerReport getLoadReport() throws Exception {
// Ensure super user access only
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 0dda3ae..902a56f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.web.RestException;
import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -170,13 +170,13 @@ public abstract class NamespacesBase extends AdminResource {
boolean isEmpty;
try {
- isEmpty = pulsar().getNamespaceService().getListOfDestinations(namespaceName).isEmpty();
+ isEmpty = pulsar().getNamespaceService().getListOfTopics(namespaceName).isEmpty();
} catch (Exception e) {
throw new RestException(e);
}
if (!isEmpty) {
- log.debug("Found destinations on namespace {}", namespaceName);
+ log.debug("Found topics on namespace {}", namespaceName);
throw new RestException(Status.CONFLICT, "Cannot delete non empty namespace");
}
@@ -269,11 +269,11 @@ public abstract class NamespacesBase extends AdminResource {
NamespaceBundle bundle = validateNamespaceBundleOwnership(namespaceName, policies.bundles, bundleRange,
authoritative, true);
try {
- List<String> destinations = pulsar().getNamespaceService().getListOfDestinations(namespaceName);
- for (String destination : destinations) {
- NamespaceBundle destinationBundle = (NamespaceBundle) pulsar().getNamespaceService()
- .getBundle(DestinationName.get(destination));
- if (bundle.equals(destinationBundle)) {
+ List<String> topics = pulsar().getNamespaceService().getListOfTopics(namespaceName);
+ for (String topic : topics) {
+ NamespaceBundle topicBundle = (NamespaceBundle) pulsar().getNamespaceService()
+ .getBundle(TopicName.get(topic));
+ if (bundle.equals(topicBundle)) {
throw new RestException(Status.CONFLICT, "Cannot delete non empty bundle");
}
}
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 5d83f8d..9386df5 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
@@ -77,8 +77,8 @@ import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
import org.apache.pulsar.common.compression.CompressionCodec;
import org.apache.pulsar.common.compression.CompressionCodecProvider;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.AuthPolicies;
@@ -126,24 +126,24 @@ public class PersistentTopicsBase extends AdminResource {
throw new RestException(e);
}
- List<String> destinations = Lists.newArrayList();
+ List<String> topics = Lists.newArrayList();
try {
String path = String.format("/managed-ledgers/%s/%s", namespaceName.toString(), domain());
- for (String destination : managedLedgerListCache().get(path)) {
- if (domain().equals(DestinationDomain.persistent.toString())) {
- destinations.add(DestinationName.get(domain(), namespaceName, decode(destination)).toString());
+ for (String topic : managedLedgerListCache().get(path)) {
+ if (domain().equals(TopicDomain.persistent.toString())) {
+ topics.add(TopicName.get(domain(), namespaceName, decode(topic)).toString());
}
}
} catch (KeeperException.NoNodeException e) {
- // NoNode means there are no destination in this domain for this namespace
+ // NoNode means there are no topics in this domain for this namespace
} catch (Exception e) {
- log.error("[{}] Failed to get destination list for namespace {}", clientAppId(), namespaceName, e);
+ log.error("[{}] Failed to get topics list for namespace {}", clientAppId(), namespaceName, e);
throw new RestException(e);
}
- destinations.sort(null);
- return destinations;
+ topics.sort(null);
+ return topics;
}
protected List<String> internalGetPartitionedTopicList() {
@@ -165,8 +165,8 @@ public class PersistentTopicsBase extends AdminResource {
try {
String partitionedTopicPath = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain());
- List<String> destinations = globalZk().getChildren(partitionedTopicPath, false);
- partitionedTopics = destinations.stream()
+ List<String> topics = globalZk().getChildren(partitionedTopicPath, false);
+ partitionedTopics = topics.stream()
.map(s -> String.format("persistent://%s/%s", namespaceName.toString(), decode(s)))
.collect(Collectors.toList());
} catch (KeeperException.NoNodeException e) {
@@ -181,11 +181,11 @@ public class PersistentTopicsBase extends AdminResource {
return partitionedTopics;
}
- protected Map<String, Set<AuthAction>> internalGetPermissionsOnDestination() {
+ protected Map<String, Set<AuthAction>> internalGetPermissionsOnTopic() {
// This operation should be reading from zookeeper and it should be allowed without having admin privileges
validateAdminAccessOnProperty(namespaceName.getProperty());
- String destinationUri = destinationName.toString();
+ String topicUri = topicName.toString();
try {
Policies policies = policiesCache().get(path(POLICIES, namespaceName.toString()))
@@ -199,17 +199,17 @@ public class PersistentTopicsBase extends AdminResource {
permissions.put(role, auth.namespace_auth.get(role));
}
- // Then add destination level permissions
- if (auth.destination_auth.containsKey(destinationUri)) {
- for (Map.Entry<String, Set<AuthAction>> entry : auth.destination_auth.get(destinationUri).entrySet()) {
+ // Then add topic level permissions
+ if (auth.destination_auth.containsKey(topicUri)) {
+ for (Map.Entry<String, Set<AuthAction>> entry : auth.destination_auth.get(topicUri).entrySet()) {
String role = entry.getKey();
- Set<AuthAction> destinationPermissions = entry.getValue();
+ Set<AuthAction> topicPermissions = entry.getValue();
if (!permissions.containsKey(role)) {
- permissions.put(role, destinationPermissions);
+ permissions.put(role, topicPermissions);
} else {
- // Do the union between namespace and destination level
- Set<AuthAction> union = Sets.union(permissions.get(role), destinationPermissions);
+ // Do the union between namespace and topic level
+ Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
permissions.put(role, union);
}
}
@@ -217,50 +217,50 @@ public class PersistentTopicsBase extends AdminResource {
return permissions;
} catch (Exception e) {
- log.error("[{}] Failed to get permissions for destination {}", clientAppId(), destinationUri, e);
+ log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicUri, e);
throw new RestException(e);
}
}
protected void validateAdminAndClientPermission() {
try {
- validateAdminAccessOnProperty(destinationName.getProperty());
+ validateAdminAccessOnProperty(topicName.getProperty());
} catch (Exception ve) {
try {
- checkAuthorization(pulsar(), destinationName, clientAppId(), clientAuthData());
+ checkAuthorization(pulsar(), topicName, clientAppId(), clientAuthData());
} catch (RestException re) {
throw re;
} catch (Exception e) {
// unknown error marked as internal server error
- log.warn("Unexpected error while authorizing request. destination={}, role={}. Error: {}",
- destinationName, clientAppId(), e.getMessage(), e);
+ log.warn("Unexpected error while authorizing request. topic={}, role={}. Error: {}",
+ topicName, clientAppId(), e.getMessage(), e);
throw new RestException(e);
}
}
}
- public void validateAdminOperationOnDestination(boolean authoritative) {
- validateAdminAccessOnProperty(destinationName.getProperty());
- validateDestinationOwnership(destinationName, authoritative);
+ public void validateAdminOperationOnTopic(boolean authoritative) {
+ validateAdminAccessOnProperty(topicName.getProperty());
+ validateTopicOwnership(topicName, authoritative);
}
- protected void internalGrantPermissionsOnDestination(String role, Set<AuthAction> actions) {
+ protected void internalGrantPermissionsOnTopic(String role, Set<AuthAction> actions) {
// This operation should be reading from zookeeper and it should be allowed without having admin privileges
validateAdminAccessOnProperty(namespaceName.getProperty());
validatePoliciesReadOnlyAccess();
- String destinationUri = destinationName.toString();
+ String topicUri = topicName.toString();
try {
Stat nodeStat = new Stat();
byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
Policies policies = jsonMapper().readValue(content, Policies.class);
- if (!policies.auth_policies.destination_auth.containsKey(destinationUri)) {
- policies.auth_policies.destination_auth.put(destinationUri, new TreeMap<String, Set<AuthAction>>());
+ if (!policies.auth_policies.destination_auth.containsKey(topicUri)) {
+ policies.auth_policies.destination_auth.put(topicUri, new TreeMap<String, Set<AuthAction>>());
}
- policies.auth_policies.destination_auth.get(destinationUri).put(role, actions);
+ policies.auth_policies.destination_auth.get(topicUri).put(role, actions);
// Write the new policies to zookeeper
globalZk().setData(path(POLICIES, namespaceName.toString()), jsonMapper().writeValueAsBytes(policies),
@@ -269,25 +269,25 @@ public class PersistentTopicsBase extends AdminResource {
// invalidate the local cache to force update
policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
- log.info("[{}] Successfully granted access for role {}: {} - destination {}", clientAppId(), role, actions,
- destinationUri);
+ log.info("[{}] Successfully granted access for role {}: {} - topic {}", clientAppId(), role, actions,
+ topicUri);
} catch (KeeperException.NoNodeException e) {
- log.warn("[{}] Failed to grant permissions on destination {}: Namespace does not exist", clientAppId(),
- destinationUri);
+ log.warn("[{}] Failed to grant permissions on topic {}: Namespace does not exist", clientAppId(),
+ topicUri);
throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
} catch (Exception e) {
- log.error("[{}] Failed to grant permissions for destination {}", clientAppId(), destinationUri, e);
+ log.error("[{}] Failed to grant permissions for topic {}", clientAppId(), topicUri, e);
throw new RestException(e);
}
}
- protected void internalRevokePermissionsOnDestination(String role) {
+ protected void internalRevokePermissionsOnTopic(String role) {
// This operation should be reading from zookeeper and it should be allowed without having admin privileges
validateAdminAccessOnProperty(namespaceName.getProperty());
validatePoliciesReadOnlyAccess();
- String destinationUri = destinationName.toString();
+ String topicUri = topicName.toString();
Stat nodeStat = new Stat();
Policies policies;
@@ -295,22 +295,22 @@ public class PersistentTopicsBase extends AdminResource {
byte[] content = globalZk().getData(path(POLICIES, namespaceName.toString()), null, nodeStat);
policies = jsonMapper().readValue(content, Policies.class);
} catch (KeeperException.NoNodeException e) {
- log.warn("[{}] Failed to revoke permissions on destination {}: Namespace does not exist", clientAppId(),
- destinationUri);
+ log.warn("[{}] Failed to revoke permissions on topic {}: Namespace does not exist", clientAppId(),
+ topicUri);
throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
} catch (Exception e) {
- log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e);
+ log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e);
throw new RestException(e);
}
- if (!policies.auth_policies.destination_auth.containsKey(destinationUri)
- || !policies.auth_policies.destination_auth.get(destinationUri).containsKey(role)) {
- log.warn("[{}] Failed to revoke permission from role {} on destination: Not set at destination level",
- clientAppId(), role, destinationUri);
- throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the destination level");
+ if (!policies.auth_policies.destination_auth.containsKey(topicUri)
+ || !policies.auth_policies.destination_auth.get(topicUri).containsKey(role)) {
+ log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level",
+ clientAppId(), role, topicUri);
+ throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the topic level");
}
- policies.auth_policies.destination_auth.get(destinationUri).remove(role);
+ policies.auth_policies.destination_auth.get(topicUri).remove(role);
try {
// Write the new policies to zookeeper
@@ -321,32 +321,32 @@ public class PersistentTopicsBase extends AdminResource {
policiesCache().invalidate(namespacePath);
globalZkCache().invalidate(namespacePath);
- log.info("[{}] Successfully revoke access for role {} - destination {}", clientAppId(), role,
- destinationUri);
+ log.info("[{}] Successfully revoke access for role {} - topic {}", clientAppId(), role,
+ topicUri);
} catch (Exception e) {
- log.error("[{}] Failed to revoke permissions for destination {}", clientAppId(), destinationUri, e);
+ log.error("[{}] Failed to revoke permissions for topic {}", clientAppId(), topicUri, e);
throw new RestException(e);
}
}
protected void internalCreatePartitionedTopic(int numPartitions, boolean authoritative) {
- validateAdminAccessOnProperty(destinationName.getProperty());
+ validateAdminAccessOnProperty(topicName.getProperty());
if (numPartitions <= 1) {
throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
}
try {
String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
- destinationName.getEncodedLocalName());
+ topicName.getEncodedLocalName());
byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
zkCreateOptimistic(path, data);
// we wait for the data to be synced in all quorums and the observers
Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
- log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+ log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
} catch (KeeperException.NodeExistsException e) {
- log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+ log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
} catch (Exception e) {
- log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+ log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
throw new RestException(e);
}
}
@@ -363,28 +363,28 @@ public class PersistentTopicsBase extends AdminResource {
* @param numPartitions
*/
protected void internalUpdatePartitionedTopic(int numPartitions) {
- validateAdminAccessOnProperty(destinationName.getProperty());
- if (destinationName.isGlobal()) {
+ validateAdminAccessOnProperty(topicName.getProperty());
+ if (topicName.isGlobal()) {
log.error("[{}] Update partitioned-topic is forbidden on global namespace {}", clientAppId(),
- destinationName);
+ topicName);
throw new RestException(Status.FORBIDDEN, "Update forbidden on global namespace");
}
if (numPartitions <= 1) {
throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
}
try {
- updatePartitionedTopic(destinationName, numPartitions).get();
+ updatePartitionedTopic(topicName, numPartitions).get();
} catch (Exception e) {
if (e.getCause() instanceof RestException) {
throw (RestException) e.getCause();
}
- log.error("[{}] Failed to update partitioned topic {}", clientAppId(), destinationName, e.getCause());
+ log.error("[{}] Failed to update partitioned topic {}", clientAppId(), topicName, e.getCause());
throw new RestException(e.getCause());
}
}
protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean authoritative) {
- PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(topicName, authoritative);
if (metadata.partitions > 1) {
validateClientVersion();
}
@@ -392,16 +392,16 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalDeletePartitionedTopic(boolean authoritative) {
- validateAdminAccessOnProperty(destinationName.getProperty());
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ validateAdminAccessOnProperty(topicName.getProperty());
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
int numPartitions = partitionMetadata.partitions;
if (numPartitions > 0) {
final CompletableFuture<Void> future = new CompletableFuture<>();
final AtomicInteger count = new AtomicInteger(numPartitions);
try {
for (int i = 0; i < numPartitions; i++) {
- DestinationName dn_partition = destinationName.getPartition(i);
- pulsar().getAdminClient().persistentTopics().deleteAsync(dn_partition.toString())
+ TopicName topicNamePartition = topicName.getPartition(i);
+ pulsar().getAdminClient().persistentTopics().deleteAsync(topicNamePartition.toString())
.whenComplete((r, ex) -> {
if (ex != null) {
if (ex instanceof NotFoundException) {
@@ -410,16 +410,16 @@ public class PersistentTopicsBase extends AdminResource {
// For all other exception, we fail the delete partition method even if a single
// partition is failed to be deleted
if (log.isDebugEnabled()) {
- log.debug("[{}] Partition not found: {}", clientAppId(), dn_partition);
+ log.debug("[{}] Partition not found: {}", clientAppId(), topicNamePartition);
}
} else {
future.completeExceptionally(ex);
- log.error("[{}] Failed to delete partition {}", clientAppId(), dn_partition,
+ log.error("[{}] Failed to delete partition {}", clientAppId(), topicNamePartition,
ex);
return;
}
} else {
- log.info("[{}] Deleted partition {}", clientAppId(), dn_partition);
+ log.info("[{}] Deleted partition {}", clientAppId(), topicNamePartition);
}
if (count.decrementAndGet() == 0) {
future.complete(null);
@@ -439,44 +439,44 @@ public class PersistentTopicsBase extends AdminResource {
// Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted
String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
- destinationName.getEncodedLocalName());
+ topicName.getEncodedLocalName());
try {
globalZk().delete(path, -1);
globalZkCache().invalidate(path);
// we wait for the data to be synced in all quorums and the observers
Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
- log.info("[{}] Deleted partitioned topic {}", clientAppId(), destinationName);
+ log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName);
} catch (KeeperException.NoNodeException nne) {
throw new RestException(Status.NOT_FOUND, "Partitioned topic does not exist");
} catch (Exception e) {
- log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), destinationName, e);
+ log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e);
throw new RestException(e);
}
}
protected void internalUnloadTopic(boolean authoritative) {
- log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
- if (destinationName.isGlobal()) {
+ log.info("[{}] Unloading topic {}", clientAppId(), topicName);
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- unloadTopic(destinationName, authoritative);
+ unloadTopic(topicName, authoritative);
}
protected void internalDeleteTopic(boolean authoritative) {
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
- if (destinationName.isGlobal()) {
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
+ if (topicName.isGlobal()) {
// Delete is disallowed on global topic
- log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), destinationName);
+ log.error("[{}] Delete topic is forbidden on global namespace {}", clientAppId(), topicName);
throw new RestException(Status.FORBIDDEN, "Delete forbidden on global namespace");
}
try {
topic.delete().get();
- log.info("[{}] Successfully removed topic {}", clientAppId(), destinationName);
+ log.info("[{}] Successfully removed topic {}", clientAppId(), topicName);
} catch (Exception e) {
Throwable t = e.getCause();
- log.error("[{}] Failed to get delete topic {}", clientAppId(), destinationName, t);
+ log.error("[{}] Failed to get delete topic {}", clientAppId(), topicName, t);
if (t instanceof TopicBusyException) {
throw new RestException(Status.PRECONDITION_FAILED, "Topic has active producers/subscriptions");
} else {
@@ -486,29 +486,29 @@ public class PersistentTopicsBase extends AdminResource {
}
protected List<String> internalGetSubscriptions(boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
List<String> subscriptions = Lists.newArrayList();
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
try {
// get the subscriptions only from the 1st partition since all the other partitions will have the same
// subscriptions
subscriptions.addAll(pulsar().getAdminClient().persistentTopics()
- .getSubscriptions(destinationName.getPartition(0).toString()));
+ .getSubscriptions(topicName.getPartition(0).toString()));
} catch (Exception e) {
throw new RestException(e);
}
} else {
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
try {
topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName));
} catch (Exception e) {
- log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), destinationName);
+ log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName);
throw new RestException(e);
}
}
@@ -518,30 +518,30 @@ public class PersistentTopicsBase extends AdminResource {
protected PersistentTopicStats internalGetStats(boolean authoritative) {
validateAdminAndClientPermission();
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- validateDestinationOwnership(destinationName, authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicOwnership(topicName, authoritative);
+ Topic topic = getTopicReference(topicName);
return topic.getStats();
}
protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) {
validateAdminAndClientPermission();
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- validateDestinationOwnership(destinationName, authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicOwnership(topicName, authoritative);
+ Topic topic = getTopicReference(topicName);
return topic.getInternalStats();
}
protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse) {
- validateAdminAccessOnProperty(destinationName.getProperty());
- if (destinationName.isGlobal()) {
+ validateAdminAccessOnProperty(topicName.getProperty());
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- String managedLedger = destinationName.getPersistenceNamingEncoding();
+ String managedLedger = topicName.getPersistenceNamingEncoding();
pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() {
@Override
public void getInfoComplete(ManagedLedgerInfo info, Object ctx) {
@@ -558,20 +558,20 @@ public class PersistentTopicsBase extends AdminResource {
}
protected PartitionedTopicStats internalGetPartitionedStats(boolean authoritative) {
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions == 0) {
throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found");
}
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata);
try {
for (int i = 0; i < partitionMetadata.partitions; i++) {
PersistentTopicStats partitionStats = pulsar().getAdminClient().persistentTopics()
- .getStats(destinationName.getPartition(i).toString());
+ .getStats(topicName.getPartition(i).toString());
stats.add(partitionStats);
- stats.partitions.put(destinationName.getPartition(i).toString(), partitionStats);
+ stats.partitions.put(topicName.getPartition(i).toString(), partitionStats);
}
} catch (Exception e) {
throw new RestException(e);
@@ -580,15 +580,15 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalDeleteSubscription(String subName, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
try {
for (int i = 0; i < partitionMetadata.partitions; i++) {
pulsar().getAdminClient().persistentTopics()
- .deleteSubscription(destinationName.getPartition(i).toString(), subName);
+ .deleteSubscription(topicName.getPartition(i).toString(), subName);
}
} catch (Exception e) {
if (e instanceof NotFoundException) {
@@ -596,18 +596,18 @@ public class PersistentTopicsBase extends AdminResource {
} else if (e instanceof PreconditionFailedException) {
throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
} else {
- log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
+ log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e);
throw new RestException(e);
}
}
} else {
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
try {
Subscription sub = topic.getSubscription(subName);
checkNotNull(sub);
sub.delete().get();
- log.info("[{}][{}] Deleted subscription {}", clientAppId(), destinationName, subName);
+ log.info("[{}][{}] Deleted subscription {}", clientAppId(), topicName, subName);
} catch (Exception e) {
Throwable t = e.getCause();
if (e instanceof NullPointerException) {
@@ -615,7 +615,7 @@ public class PersistentTopicsBase extends AdminResource {
} else if (t instanceof SubscriptionBusyException) {
throw new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers");
} else {
- log.error("[{}] Failed to delete subscription {} {}", clientAppId(), destinationName, subName, e);
+ log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e);
throw new RestException(t);
}
}
@@ -623,22 +623,22 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalSkipAllMessages(String subName, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
try {
for (int i = 0; i < partitionMetadata.partitions; i++) {
pulsar().getAdminClient().persistentTopics()
- .skipAllMessages(destinationName.getPartition(i).toString(), subName);
+ .skipAllMessages(topicName.getPartition(i).toString(), subName);
}
} catch (Exception e) {
throw new RestException(e);
}
} else {
- validateAdminOperationOnDestination(authoritative);
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
try {
if (subName.startsWith(topic.replicatorPrefix)) {
String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -650,26 +650,26 @@ public class PersistentTopicsBase extends AdminResource {
checkNotNull(sub);
sub.clearBacklog().get();
}
- log.info("[{}] Cleared backlog on {} {}", clientAppId(), destinationName, subName);
+ log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName);
} catch (NullPointerException npe) {
throw new RestException(Status.NOT_FOUND, "Subscription not found");
} catch (Exception exception) {
- log.error("[{}] Failed to skip all messages {} {}", clientAppId(), destinationName, subName, exception);
+ log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, exception);
throw new RestException(exception);
}
}
}
protected void internalSkipMessages(String subName, int numMessages, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
throw new RestException(Status.METHOD_NOT_ALLOWED, "Skip messages on a partitioned topic is not allowed");
}
- validateAdminOperationOnDestination(authoritative);
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
try {
if (subName.startsWith(topic.replicatorPrefix)) {
String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -681,37 +681,37 @@ public class PersistentTopicsBase extends AdminResource {
checkNotNull(sub);
sub.skipMessages(numMessages).get();
}
- log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, destinationName, subName);
+ log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, topicName, subName);
} catch (NullPointerException npe) {
throw new RestException(Status.NOT_FOUND, "Subscription not found");
} catch (Exception exception) {
- log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, destinationName, subName,
+ log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName,
exception);
throw new RestException(exception);
}
}
protected void internalExpireMessagesForAllSubscriptions(int expireTimeInSeconds, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
try {
- // expire messages for each partition destination
+ // expire messages for each partition topic
for (int i = 0; i < partitionMetadata.partitions; i++) {
pulsar().getAdminClient().persistentTopics().expireMessagesForAllSubscriptions(
- destinationName.getPartition(i).toString(), expireTimeInSeconds);
+ topicName.getPartition(i).toString(), expireTimeInSeconds);
}
} catch (Exception e) {
log.error("[{}] Failed to expire messages up to {} on {} {}", clientAppId(), expireTimeInSeconds,
- destinationName, e);
+ topicName, e);
throw new RestException(e);
}
} else {
// validate ownership and redirect if current broker is not owner
- validateAdminOperationOnDestination(authoritative);
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
topic.getReplicators().forEach((subName, replicator) -> {
internalExpireMessages(subName, expireTimeInSeconds, authoritative);
});
@@ -722,10 +722,10 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalResetCursor(String subName, long timestamp, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
int numParts = partitionMetadata.partitions;
@@ -733,7 +733,7 @@ public class PersistentTopicsBase extends AdminResource {
Exception partitionException = null;
try {
for (int i = 0; i < numParts; i++) {
- pulsar().getAdminClient().persistentTopics().resetCursor(destinationName.getPartition(i).toString(),
+ pulsar().getAdminClient().persistentTopics().resetCursor(topicName.getPartition(i).toString(),
subName, timestamp);
}
} catch (PreconditionFailedException pfe) {
@@ -743,24 +743,24 @@ public class PersistentTopicsBase extends AdminResource {
partitionException = pfe;
} catch (Exception e) {
log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
- destinationName, subName, timestamp, e);
+ topicName, subName, timestamp, e);
throw new RestException(e);
}
// report an error to user if unable to reset for all partitions
if (numPartException == numParts) {
log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
- destinationName, subName, timestamp, partitionException);
+ topicName, subName, timestamp, partitionException);
throw new RestException(Status.PRECONDITION_FAILED, partitionException.getMessage());
} else if (numPartException > 0) {
log.warn("[{}][{}] partial errors for reset cursor on subscription {} to time {} - ", clientAppId(),
- destinationName, subName, timestamp, partitionException);
+ topicName, subName, timestamp, partitionException);
}
} else {
- validateAdminOperationOnDestination(authoritative);
- log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), destinationName,
+ validateAdminOperationOnTopic(authoritative);
+ log.info("[{}][{}] received reset cursor on subscription {} to time {}", clientAppId(), topicName,
subName, timestamp);
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
if (topic == null) {
throw new RestException(Status.NOT_FOUND, "Topic not found");
}
@@ -768,12 +768,12 @@ public class PersistentTopicsBase extends AdminResource {
PersistentSubscription sub = topic.getSubscription(subName);
checkNotNull(sub);
sub.resetCursor(timestamp).get();
- log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), destinationName, subName,
+ log.info("[{}][{}] reset cursor on subscription {} to time {}", clientAppId(), topicName, subName,
timestamp);
} catch (Exception e) {
Throwable t = e.getCause();
log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(),
- destinationName, subName, timestamp, e);
+ topicName, subName, timestamp, e);
if (e instanceof NullPointerException) {
throw new RestException(Status.NOT_FOUND, "Subscription not found");
} else if (e instanceof NotAllowedException) {
@@ -789,13 +789,13 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalCreateSubscription(String subscriptionName, MessageIdImpl messageId, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), destinationName,
+ log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), topicName,
subscriptionName, messageId);
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
try {
if (partitionMetadata.partitions > 0) {
@@ -805,15 +805,15 @@ public class PersistentTopicsBase extends AdminResource {
for (int i = 0; i < partitionMetadata.partitions; i++) {
futures.add(admin.persistentTopics().createSubscriptionAsync(
- destinationName.getPartition(i).toString(),
+ topicName.getPartition(i).toString(),
subscriptionName, messageId));
}
FutureUtil.waitForAll(futures).join();
} else {
- validateAdminOperationOnDestination(authoritative);
+ validateAdminOperationOnTopic(authoritative);
- PersistentTopic topic = (PersistentTopic) getOrCreateTopic(destinationName);
+ PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName);
if (topic.getSubscriptions().containsKey(subscriptionName)) {
throw new RestException(Status.CONFLICT, "Subscription already exists for topic");
@@ -823,12 +823,12 @@ public class PersistentTopicsBase extends AdminResource {
.createSubscription(subscriptionName).get();
subscription.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(),
- destinationName, subscriptionName, messageId);
+ topicName, subscriptionName, messageId);
}
} catch (Exception e) {
Throwable t = e.getCause();
log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(),
- destinationName, subscriptionName, messageId, e);
+ topicName, subscriptionName, messageId, e);
if (t instanceof SubscriptionInvalidCursorPosition) {
throw new RestException(Status.PRECONDITION_FAILED,
"Unable to find position for position specified: " + t.getMessage());
@@ -839,22 +839,22 @@ public class PersistentTopicsBase extends AdminResource {
}
protected void internalResetCursorOnPosition(String subName, boolean authoritative, MessageIdImpl messageId) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), destinationName,
+ log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), topicName,
subName, messageId);
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
- log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), destinationName,
+ log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), topicName,
subName);
throw new RestException(Status.METHOD_NOT_ALLOWED,
"Reset-cursor at position is not allowed for partitioned-topic");
} else {
- validateAdminOperationOnDestination(authoritative);
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
if (topic == null) {
throw new RestException(Status.NOT_FOUND, "Topic not found");
}
@@ -863,11 +863,11 @@ public class PersistentTopicsBase extends AdminResource {
checkNotNull(sub);
sub.resetCursor(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId())).get();
log.info("[{}][{}] successfully reset cursor on subscription {} to position {}", clientAppId(),
- destinationName, subName, messageId);
+ topicName, subName, messageId);
} catch (Exception e) {
Throwable t = e.getCause();
log.warn("[{}] [{}] Failed to reset cursor on subscription {} to position {}", clientAppId(),
- destinationName, subName, messageId, e);
+ topicName, subName, messageId, e);
if (e instanceof NullPointerException) {
throw new RestException(Status.NOT_FOUND, "Subscription not found");
} else if (t instanceof SubscriptionInvalidCursorPosition) {
@@ -881,21 +881,21 @@ public class PersistentTopicsBase extends AdminResource {
}
protected Response internalPeekNthMessage(String subName, int messagePosition, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
throw new RestException(Status.METHOD_NOT_ALLOWED, "Peek messages on a partitioned topic is not allowed");
}
- validateAdminOperationOnDestination(authoritative);
- if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
- log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+ validateAdminOperationOnTopic(authoritative);
+ if (!(getTopicReference(topicName) instanceof PersistentTopic)) {
+ log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), topicName,
subName);
throw new RestException(Status.METHOD_NOT_ALLOWED,
"Skip messages on a non-persistent topic is not allowed");
}
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
PersistentReplicator repl = null;
PersistentSubscription sub = null;
Entry entry = null;
@@ -956,7 +956,7 @@ public class PersistentTopicsBase extends AdminResource {
throw new RestException(Status.NOT_FOUND, "Message not found");
} catch (Exception exception) {
log.error("[{}] Failed to get message at position {} from {} {}", clientAppId(), messagePosition,
- destinationName, subName, exception);
+ topicName, subName, exception);
throw new RestException(exception);
} finally {
if (entry != null) {
@@ -966,11 +966,11 @@ public class PersistentTopicsBase extends AdminResource {
}
protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
// Validate that namespace exists, throw 404 if it doesn't exist
- // note that we do not want to load the topic and hence skip validateAdminOperationOnDestination()
+ // note that we do not want to load the topic and hence skip validateAdminOperationOnTopic()
try {
policiesCache().get(path(POLICIES, namespaceName.toString()));
} catch (KeeperException.NoNodeException e) {
@@ -984,7 +984,7 @@ public class PersistentTopicsBase extends AdminResource {
PersistentOfflineTopicStats offlineTopicStats = null;
try {
- offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(destinationName);
+ offlineTopicStats = pulsar().getBrokerService().getOfflineTopicStat(topicName);
if (offlineTopicStats != null) {
// offline topic stat has a cost - so use cached value until TTL
long elapsedMs = System.currentTimeMillis() - offlineTopicStats.statGeneratedAt.getTime();
@@ -992,13 +992,13 @@ public class PersistentTopicsBase extends AdminResource {
return offlineTopicStats;
}
}
- final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(destinationName)
+ final ManagedLedgerConfig config = pulsar().getBrokerService().getManagedLedgerConfig(topicName)
.get();
ManagedLedgerOfflineBacklog offlineTopicBacklog = new ManagedLedgerOfflineBacklog(config.getDigestType(),
config.getPassword(), pulsar().getAdvertisedAddress(), false);
offlineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog(
- (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), destinationName);
- pulsar().getBrokerService().cacheOfflineTopicStats(destinationName, offlineTopicStats);
+ (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), topicName);
+ pulsar().getBrokerService().cacheOfflineTopicStats(topicName, offlineTopicStats);
} catch (Exception exception) {
throw new RestException(exception);
}
@@ -1006,48 +1006,48 @@ public class PersistentTopicsBase extends AdminResource {
}
protected MessageId internalTerminate(boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a partitioned topic is not allowed");
}
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
try {
return ((PersistentTopic) topic).terminate().get();
} catch (Exception exception) {
- log.error("[{}] Failed to terminated topic {}", clientAppId(), destinationName, exception);
+ log.error("[{}] Failed to terminated topic {}", clientAppId(), topicName, exception);
throw new RestException(exception);
}
}
protected void internalExpireMessages(String subName, int expireTimeInSeconds, boolean authoritative) {
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(destinationName, authoritative);
+ PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative);
if (partitionMetadata.partitions > 0) {
- // expire messages for each partition destination
+ // expire messages for each partition topic
try {
for (int i = 0; i < partitionMetadata.partitions; i++) {
pulsar().getAdminClient().persistentTopics()
- .expireMessages(destinationName.getPartition(i).toString(), subName, expireTimeInSeconds);
+ .expireMessages(topicName.getPartition(i).toString(), subName, expireTimeInSeconds);
}
} catch (Exception e) {
throw new RestException(e);
}
} else {
// validate ownership and redirect if current broker is not owner
- validateAdminOperationOnDestination(authoritative);
- if (!(getTopicReference(destinationName) instanceof PersistentTopic)) {
- log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), destinationName,
+ validateAdminOperationOnTopic(authoritative);
+ if (!(getTopicReference(topicName) instanceof PersistentTopic)) {
+ log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), topicName,
subName);
throw new RestException(Status.METHOD_NOT_ALLOWED,
"Expire messages on a non-persistent topic is not allowed");
}
- PersistentTopic topic = (PersistentTopic) getTopicReference(destinationName);
+ PersistentTopic topic = (PersistentTopic) getTopicReference(topicName);
try {
if (subName.startsWith(topic.replicatorPrefix)) {
String remoteCluster = PersistentReplicator.getRemoteCluster(subName);
@@ -1060,50 +1060,50 @@ public class PersistentTopicsBase extends AdminResource {
sub.expireMessages(expireTimeInSeconds);
}
log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), expireTimeInSeconds,
- destinationName, subName);
+ topicName, subName);
} catch (NullPointerException npe) {
throw new RestException(Status.NOT_FOUND, "Subscription not found");
} catch (Exception exception) {
log.error("[{}] Failed to expire messages up to {} on {} with subscription {} {}", clientAppId(),
- expireTimeInSeconds, destinationName, subName, exception);
+ expireTimeInSeconds, topicName, subName, exception);
throw new RestException(exception);
}
}
}
public static CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(PulsarService pulsar,
- String clientAppId, AuthenticationDataSource authenticationData, DestinationName dn) {
+ String clientAppId, AuthenticationDataSource authenticationData, TopicName topicName) {
CompletableFuture<PartitionedTopicMetadata> metadataFuture = new CompletableFuture<>();
try {
// (1) authorize client
try {
- checkAuthorization(pulsar, dn, clientAppId, authenticationData);
+ checkAuthorization(pulsar, topicName, clientAppId, authenticationData);
} catch (RestException e) {
try {
- validateAdminAccessOnProperty(pulsar, clientAppId, dn.getProperty());
+ validateAdminAccessOnProperty(pulsar, clientAppId, topicName.getProperty());
} catch (RestException authException) {
- log.warn("Failed to authorize {} on cluster {}", clientAppId, dn.toString());
+ log.warn("Failed to authorize {} on cluster {}", clientAppId, topicName.toString());
throw new PulsarClientException(String.format("Authorization failed %s on topic %s with error %s",
- clientAppId, dn.toString(), authException.getMessage()));
+ clientAppId, topicName.toString(), authException.getMessage()));
}
} catch (Exception ex) {
// throw without wrapping to PulsarClientException that considers: unknown error marked as internal
// server error
log.warn("Failed to authorize {} on cluster {} with unexpected exception {}", clientAppId,
- dn.toString(), ex.getMessage(), ex);
+ topicName.toString(), ex.getMessage(), ex);
throw ex;
}
- String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getNamespace(), dn.getDomain().toString(),
- dn.getEncodedLocalName());
+ String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getNamespace(), topicName.getDomain().toString(),
+ topicName.getEncodedLocalName());
// validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can
// serve/redirect request else fail partitioned-metadata-request so, client fails while creating
// producer/consumer
- checkLocalOrGetPeerReplicationCluster(pulsar, dn.getNamespaceObject())
+ checkLocalOrGetPeerReplicationCluster(pulsar, topicName.getNamespaceObject())
.thenCompose(res -> fetchPartitionedTopicMetadataAsync(pulsar, path)).thenAccept(metadata -> {
if (log.isDebugEnabled()) {
- log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, dn,
+ log.debug("[{}] Total number of partitions for topic {} is {}", clientAppId, topicName,
metadata.partitions);
}
metadataFuture.complete(metadata);
@@ -1120,9 +1120,9 @@ public class PersistentTopicsBase extends AdminResource {
/**
* Get the Topic object reference from the Pulsar broker
*/
- private Topic getTopicReference(DestinationName dn) {
+ private Topic getTopicReference(TopicName topicName) {
try {
- Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+ Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
checkNotNull(topic);
return topic;
} catch (Exception e) {
@@ -1130,9 +1130,9 @@ public class PersistentTopicsBase extends AdminResource {
}
}
- private Topic getOrCreateTopic(DestinationName dn) {
+ private Topic getOrCreateTopic(TopicName topicName) {
try {
- return pulsar().getBrokerService().getTopic(dn.toString()).get();
+ return pulsar().getBrokerService().getTopic(topicName.toString()).get();
} catch (InterruptedException | ExecutionException e) {
throw new RestException(e);
}
@@ -1163,12 +1163,12 @@ public class PersistentTopicsBase extends AdminResource {
}
}
- private CompletableFuture<Void> updatePartitionedTopic(DestinationName dn, int numPartitions) {
- String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(),
- domain(), dn.getEncodedLocalName());
+ private CompletableFuture<Void> updatePartitionedTopic(TopicName topicName, int numPartitions) {
+ String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getProperty(), topicName.getCluster(), topicName.getNamespacePortion(),
+ domain(), topicName.getEncodedLocalName());
CompletableFuture<Void> updatePartition = new CompletableFuture<>();
- createSubscriptions(dn, numPartitions).thenAccept(res -> {
+ createSubscriptions(topicName, numPartitions).thenAccept(res -> {
try {
byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
globalZk().setData(path, data, -1, (rc, path1, ctx, stat) -> {
@@ -1193,14 +1193,14 @@ public class PersistentTopicsBase extends AdminResource {
/**
* It creates subscriptions for new partitions of existing partitioned-topics
*
- * @param dn
+ * @param topicName
* : topic-name: persistent://prop/cluster/ns/topic
* @param numPartitions
* : number partitions for the topics
*/
- private CompletableFuture<Void> createSubscriptions(DestinationName dn, int numPartitions) {
- String path = path(PARTITIONED_TOPIC_PATH_ZNODE, dn.getProperty(), dn.getCluster(), dn.getNamespacePortion(),
- domain(), dn.getEncodedLocalName());
+ private CompletableFuture<Void> createSubscriptions(TopicName topicName, int numPartitions) {
+ String path = path(PARTITIONED_TOPIC_PATH_ZNODE, topicName.getProperty(), topicName.getCluster(),
+ topicName.getNamespacePortion(), domain(), topicName.getEncodedLocalName());
CompletableFuture<Void> result = new CompletableFuture<>();
fetchPartitionedTopicMetadataAsync(pulsar(), path).thenAccept(partitionMetadata -> {
if (partitionMetadata.partitions <= 1) {
@@ -1222,21 +1222,21 @@ public class PersistentTopicsBase extends AdminResource {
return;
}
- admin.persistentTopics().getStatsAsync(dn.getPartition(0).toString()).thenAccept(stats -> {
+ admin.persistentTopics().getStatsAsync(topicName.getPartition(0).toString()).thenAccept(stats -> {
stats.subscriptions.keySet().forEach(subscription -> {
List<CompletableFuture<Void>> subscriptionFutures = new ArrayList<>();
for (int i = partitionMetadata.partitions; i < numPartitions; i++) {
- final String topicName = dn.getPartition(i).toString();
+ final String topicNamePartition = topicName.getPartition(i).toString();
- subscriptionFutures.add(admin.persistentTopics().createSubscriptionAsync(topicName,
+ subscriptionFutures.add(admin.persistentTopics().createSubscriptionAsync(topicNamePartition,
subscription, MessageId.latest));
}
FutureUtil.waitForAll(subscriptionFutures).thenRun(() -> {
- log.info("[{}] Successfully created new partitions {}", clientAppId(), dn);
+ log.info("[{}] Successfully created new partitions {}", clientAppId(), topicName);
result.complete(null);
}).exceptionally(ex -> {
- log.warn("[{}] Failed to create subscriptions on new partitions for {}", clientAppId(), dn, ex);
+ log.warn("[{}] Failed to create subscriptions on new partitions for {}", clientAppId(), topicName, ex);
result.completeExceptionally(ex);
return null;
});
@@ -1246,31 +1246,31 @@ public class PersistentTopicsBase extends AdminResource {
// The first partition doesn't exist, so there are currently to subscriptions to recreate
result.complete(null);
} else {
- log.warn("[{}] Failed to get list of subscriptions of {}", clientAppId(), dn.getPartition(0), ex);
+ log.warn("[{}] Failed to get list of subscriptions of {}", clientAppId(), topicName.getPartition(0), ex);
result.completeExceptionally(ex);
}
return null;
});
}).exceptionally(ex -> {
- log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), dn.toString());
+ log.warn("[{}] Failed to get partition metadata for {}", clientAppId(), topicName.toString());
result.completeExceptionally(ex);
return null;
});
return result;
}
- protected void unloadTopic(DestinationName destination, boolean authoritative) {
+ protected void unloadTopic(TopicName topicName, boolean authoritative) {
validateSuperUserAccess();
- validateDestinationOwnership(destination, authoritative);
+ validateTopicOwnership(topicName, authoritative);
try {
- Topic topic = getTopicReference(destination);
+ Topic topic = getTopicReference(topicName);
topic.close().get();
- log.info("[{}] Successfully unloaded topic {}", clientAppId(), destination);
+ log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName);
} catch (NullPointerException e) {
- log.error("[{}] topic {} not found", clientAppId(), destination);
+ log.error("[{}] topic {} not found", clientAppId(), topicName);
throw new RestException(Status.NOT_FOUND, "Topic does not exist");
} catch (Exception e) {
- log.error("[{}] Failed to unload topic {}, {}", clientAppId(), destination, e.getCause().getMessage(), e);
+ log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, e.getCause().getMessage(), e);
throw new RestException(e.getCause());
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
index 6845e91..4cb6d5f 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PropertiesBase.java
@@ -161,7 +161,7 @@ public class PropertiesBase extends AdminResource {
@DELETE
@Path("/{property}")
- @ApiOperation(value = "elete a property and all namespaces and destinations under it.")
+ @ApiOperation(value = "elete a property and all namespaces and topics under it.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Property does not exist"),
@ApiResponse(code = 409, message = "The property still has active namespaces") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
index 3ae1d4b..ddd3154 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java
@@ -69,6 +69,7 @@ import io.swagger.annotations.ApiResponses;
@Produces(MediaType.APPLICATION_JSON)
@Consumes(MediaType.APPLICATION_JSON)
@Api(value = "/namespaces", description = "Namespaces admin apis", tags = "namespaces")
+@SuppressWarnings("deprecation")
public class Namespaces extends NamespacesBase {
@GET
@@ -112,10 +113,10 @@ public class Namespaces extends NamespacesBase {
@GET
@Path("/{property}/{cluster}/{namespace}/destinations")
- @ApiOperation(hidden = true, value = "Get the list of all the destinations under a certain namespace.", response = String.class, responseContainer = "Set")
+ @ApiOperation(hidden = true, value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
- public List<String> getDestinations(@PathParam("property") String property,
+ public List<String> getTopics(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace) {
validateAdminAccessOnProperty(property);
validateNamespaceName(property, cluster, namespace);
@@ -124,7 +125,7 @@ public class Namespaces extends NamespacesBase {
getNamespacePolicies(namespaceName);
try {
- return pulsar().getNamespaceService().getListOfDestinations(namespaceName);
+ return pulsar().getNamespaceService().getListOfTopics(namespaceName);
} catch (Exception e) {
log.error("Failed to get topics list for namespace {}/{}/{}", property, cluster, namespace, e);
throw new RestException(e);
@@ -178,7 +179,7 @@ public class Namespaces extends NamespacesBase {
@DELETE
@Path("/{property}/{cluster}/{namespace}")
- @ApiOperation(hidden = true, value = "Delete a namespace and all the destinations under it.")
+ @ApiOperation(hidden = true, value = "Delete a namespace and all the topics under it.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
@ApiResponse(code = 409, message = "Namespace is not empty") })
@@ -191,7 +192,7 @@ public class Namespaces extends NamespacesBase {
@DELETE
@Path("/{property}/{cluster}/{namespace}/{bundle}")
- @ApiOperation(hidden = true, value = "Delete a namespace bundle and all the destinations under it.")
+ @ApiOperation(hidden = true, value = "Delete a namespace bundle and all the topics under it.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"),
@ApiResponse(code = 409, message = "Namespace bundle is not empty") })
@@ -460,7 +461,7 @@ public class Namespaces extends NamespacesBase {
@PUT
@Path("/{property}/{cluster}/{namespace}/unload")
@ApiOperation(hidden = true, value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
- + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+ + "removes all producers, consumers, and connections using this namespace, and close all topics (including"
+ "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
+ "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
+ "result in non-persistent message loss and unexpected connection closure to the clients.")
@@ -533,7 +534,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/backlogQuota")
- @ApiOperation(hidden = true, value = " Set a backlog quota for all the destinations on a namespace.")
+ @ApiOperation(hidden = true, value = " Set a backlog quota for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification"),
@@ -584,7 +585,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/persistence")
- @ApiOperation(hidden = true, value = "Set the persistence configuration for all the destinations on a namespace.")
+ @ApiOperation(hidden = true, value = "Set the persistence configuration for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification"),
@@ -609,7 +610,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/clearBacklog")
- @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace.")
+ @ApiOperation(hidden = true, value = "Clear backlog for all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -621,7 +622,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog")
- @ApiOperation(hidden = true, value = "Clear backlog for all destinations on a namespace bundle.")
+ @ApiOperation(hidden = true, value = "Clear backlog for all topics on a namespace bundle.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBundleBacklog(@PathParam("property") String property,
@@ -634,7 +635,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/clearBacklog/{subscription}")
- @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace.")
+ @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
@@ -647,7 +648,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/{bundle}/clearBacklog/{subscription}")
- @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+ @ApiOperation(hidden = true, value = "Clear backlog for a given subscription on all topics on a namespace bundle.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
@@ -660,7 +661,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/unsubscribe/{subscription}")
- @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace.")
+ @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -672,7 +673,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/{bundle}/unsubscribe/{subscription}")
- @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all destinations on a namespace bundle.")
+ @ApiOperation(hidden = true, value = "Unsubscribes the given subscription on all topics on a namespace bundle.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void unsubscribeNamespaceBundle(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -685,7 +686,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{cluster}/{namespace}/subscriptionAuthMode")
- @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+ @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
index c423f21..0f9ffe1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java
@@ -37,7 +37,7 @@ import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.broker.web.RestException;
import org.apache.pulsar.common.naming.Constants;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
@@ -63,96 +63,97 @@ import java.util.concurrent.ExecutionException;
@Path("/non-persistent")
@Produces(MediaType.APPLICATION_JSON)
@Api(value = "/non-persistent", description = "Non-Persistent topic admin apis", tags = "non-persistent topic")
+@SuppressWarnings("deprecation")
public class NonPersistentTopics extends PersistentTopics {
private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
@GET
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- return getPartitionedTopicMetadata(destinationName, authoritative);
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ return getPartitionedTopicMetadata(topicName, authoritative);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/stats")
+ @Path("{property}/{cluster}/{namespace}/{topic}/stats")
@ApiOperation(hidden = true, value = "Get the stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public NonPersistentTopicStats getStats(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
return ((NonPersistentTopic) topic).getStats();
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
+ @Path("{property}/{cluster}/{namespace}/{topic}/internalStats")
@ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- validateAdminOperationOnDestination(authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ validateAdminOperationOnTopic(authoritative);
+ Topic topic = getTopicReference(topicName);
return topic.getInternalStats();
}
@PUT
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic already exist") })
public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- validateAdminAccessOnProperty(destinationName.getProperty());
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ validateAdminAccessOnProperty(topicName.getProperty());
if (numPartitions <= 1) {
throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
}
try {
String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
- destinationName.getEncodedLocalName());
+ topicName.getEncodedLocalName());
byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
zkCreateOptimistic(path, data);
// we wait for the data to be synced in all quorums and the observers
Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
- log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+ log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
} catch (KeeperException.NodeExistsException e) {
- log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+ log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
} catch (Exception e) {
- log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+ log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
throw new RestException(e);
}
}
@PUT
- @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/unload")
@ApiOperation(hidden = true, value = "Unload a topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ log.info("[{}] Unloading topic {}", clientAppId(), topicName);
- if (destinationName.isGlobal()) {
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- unloadTopic(destinationName, authoritative);
+ unloadTopic(topicName, authoritative);
}
@GET
@@ -236,7 +237,7 @@ public class NonPersistentTopics extends PersistentTopics {
try {
final List<String> topicList = Lists.newArrayList();
pulsar().getBrokerService().getTopics().forEach((name, topicFuture) -> {
- DestinationName topicName = DestinationName.get(name);
+ TopicName topicName = TopicName.get(name);
if (nsBundle.includes(topicName)) {
topicList.add(name);
}
@@ -248,14 +249,14 @@ public class NonPersistentTopics extends PersistentTopics {
}
}
- protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
- validateAdminAccessOnProperty(fqdn.getProperty());
- validateDestinationOwnership(fqdn, authoritative);
+ protected void validateAdminOperationOnTopic(TopicName topicName, boolean authoritative) {
+ validateAdminAccessOnProperty(topicName.getProperty());
+ validateTopicOwnership(topicName, authoritative);
}
- private Topic getTopicReference(DestinationName dn) {
+ private Topic getTopicReference(TopicName topicName) {
try {
- Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+ Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
checkNotNull(topic);
return topic;
} catch (Exception e) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
index 624a3f3..8e10542 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java
@@ -57,7 +57,9 @@ import io.swagger.annotations.ApiResponses;
@Path("/persistent")
@Produces(MediaType.APPLICATION_JSON)
@Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic")
+@SuppressWarnings("deprecation")
public class PersistentTopics extends PersistentTopicsBase {
+
@GET
@Path("/{property}/{cluster}/{namespace}")
@ApiOperation(hidden = true, value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List")
@@ -81,55 +83,55 @@ public class PersistentTopics extends PersistentTopicsBase {
}
@GET
- @Path("/{property}/{cluster}/{namespace}/{destination}/permissions")
- @ApiOperation(hidden = true, value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
- + "namespace level combined (union) with any eventual specific permission set on the destination.")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/permissions")
+ @ApiOperation(hidden = true, value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the"
+ + "namespace level combined (union) with any eventual specific permission set on the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist") })
- public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
+ public Map<String, Set<AuthAction>> getPermissionsOnTopic(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- return internalGetPermissionsOnDestination();
+ @PathParam("topic") @Encoded String encodedTopic) {
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ return internalGetPermissionsOnTopic();
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/permissions/{role}")
@ApiOperation(hidden = true, value = "Grant a new permission to a role on a single topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist"),
@ApiResponse(code = 409, message = "Concurrent modification") })
- public void grantPermissionsOnDestination(@PathParam("property") String property,
+ public void grantPermissionsOnTopic(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("role") String role,
Set<AuthAction> actions) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- internalGrantPermissionsOnDestination(role, actions);
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ internalGrantPermissionsOnTopic(role, actions);
}
@DELETE
- @Path("/{property}/{cluster}/{namespace}/{destination}/permissions/{role}")
- @ApiOperation(hidden = true, value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+ @Path("/{property}/{cluster}/{namespace}/{topic}/permissions/{role}")
+ @ApiOperation(hidden = true, value = "Revoke permissions on a topic.", notes = "Revoke permissions to a role on a single topic. If the permission was not set at the topic"
+ "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist"),
- @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
- public void revokePermissionsOnDestination(@PathParam("property") String property,
+ @ApiResponse(code = 412, message = "Permissions are not set at the topic level") })
+ public void revokePermissionsOnTopic(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("role") String role) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
- internalRevokePermissionsOnDestination(role);
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("role") String role) {
+ validateTopicName(property, cluster, namespace, encodedTopic);
+ internalRevokePermissionsOnTopic(role);
}
@PUT
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic already exist") })
public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
int numPartitions, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalCreatePartitionedTopic(numPartitions, authoritative);
}
@@ -148,274 +150,274 @@ public class PersistentTopics extends PersistentTopicsBase {
* @param numPartitions
*/
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic does not exist") })
public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
int numPartitions) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalUpdatePartitionedTopic(numPartitions);
}
@GET
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Get partitioned topic metadata.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetPartitionedMetadata(authoritative);
}
@DELETE
- @Path("/{property}/{cluster}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/partitions")
@ApiOperation(hidden = true, value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Partitioned topic does not exist") })
public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalDeletePartitionedTopic(authoritative);
}
@PUT
- @Path("/{property}/{cluster}/{namespace}/{destination}/unload")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/unload")
@ApiOperation(hidden = true, value = "Unload a topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalUnloadTopic(authoritative);
}
@DELETE
- @Path("/{property}/{cluster}/{namespace}/{destination}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}")
@ApiOperation(hidden = true, value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist"),
@ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
public void deleteTopic(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalDeleteTopic(authoritative);
}
@GET
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscriptions")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscriptions")
@ApiOperation(hidden = true, value = "Get the list of persistent subscriptions for a given topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public List<String> getSubscriptions(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetSubscriptions(authoritative);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/stats")
+ @Path("{property}/{cluster}/{namespace}/{topic}/stats")
@ApiOperation(hidden = true, value = "Get the stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicStats getStats(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetStats(authoritative);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/internalStats")
+ @Path("{property}/{cluster}/{namespace}/{topic}/internalStats")
@ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetInternalStats(authoritative);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/internal-info")
+ @Path("{property}/{cluster}/{namespace}/{topic}/internal-info")
@ApiOperation(hidden = true, value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@Suspended AsyncResponse asyncResponse) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalGetManagedLedgerInfo(asyncResponse);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/partitioned-stats")
+ @Path("{property}/{cluster}/{namespace}/{topic}/partitioned-stats")
@ApiOperation(hidden = true, value = "Get the stats for the partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetPartitionedStats(authoritative);
}
@DELETE
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}")
@ApiOperation(hidden = true, value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist"),
@ApiResponse(code = 412, message = "Subscription has active consumers") })
public void deleteSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalDeleteSubscription(subName, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip_all")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skip_all")
@ApiOperation(hidden = true, value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void skipAllMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalSkipAllMessages(subName, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}")
@ApiOperation(hidden = true, value = "Skip messages on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void skipMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName, @PathParam("numMessages") int numMessages,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalSkipMessages(subName, numMessages, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
@ApiOperation(hidden = true, value = "Expire messages on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void expireTopicMessages(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName, @PathParam("expireTimeInSeconds") int expireTimeInSeconds,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalExpireMessages(subName, expireTimeInSeconds, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}")
@ApiOperation(hidden = true, value = "Expire messages on all subscriptions of topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@PathParam("expireTimeInSeconds") int expireTimeInSeconds,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/resetcursor/{timestamp}")
@ApiOperation(hidden = true, value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
public void resetCursor(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName, @PathParam("timestamp") long timestamp,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalResetCursor(subName, timestamp, authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/resetcursor")
@ApiOperation(hidden = true, value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
@ApiResponse(code = 405, message = "Not supported for partitioned topics") })
public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
internalResetCursorOnPosition(subName, authoritative, messageId);
}
@PUT
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subscriptionName}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subscriptionName}")
@ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "Creates a subscription on the topic at the specified message id")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
@ApiResponse(code = 405, message = "Not supported for partitioned topics") })
public void createSubscription(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String destination,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String topic,
@PathParam("subscriptionName") String subscriptionName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
- validateDestinationName(property, cluster, namespace, destination);
+ validateTopicName(property, cluster, namespace, topic);
internalCreateSubscription(subscriptionName, messageId, authoritative);
}
@GET
- @Path("/{property}/{cluster}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}")
@ApiOperation(hidden = true, value = "Peek nth message on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
public Response peekNthMessage(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("subName") String subName, @PathParam("messagePosition") int messagePosition,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalPeekNthMessage(subName, messagePosition, authoritative);
}
@GET
- @Path("{property}/{cluster}/{namespace}/{destination}/backlog")
+ @Path("{property}/{cluster}/{namespace}/{topic}/backlog")
@ApiOperation(hidden = true, value = "Get estimated backlog for offline topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
@PathParam("cluster") String cluster, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalGetBacklog(authoritative);
}
@POST
- @Path("/{property}/{cluster}/{namespace}/{destination}/terminate")
+ @Path("/{property}/{cluster}/{namespace}/{topic}/terminate")
@ApiOperation(hidden = true, value = "Terminate a topic. A topic that is terminated will not accept any more "
+ "messages to be published and will let consumer to drain existing messages in backlog")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public MessageId terminate(@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, cluster, namespace, encodedTopic);
+ validateTopicName(property, cluster, namespace, encodedTopic);
return internalTerminate(authoritative);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index d00499a..873d761 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -73,7 +73,7 @@ public class Namespaces extends NamespacesBase {
@ApiOperation(value = "Get the list of all the topics under a certain namespace.", response = String.class, responseContainer = "Set")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") })
- public List<String> getDestinations(@PathParam("property") String property,
+ public List<String> getTopics(@PathParam("property") String property,
@PathParam("namespace") String namespace) {
validateAdminAccessOnProperty(property);
validateNamespaceName(property, namespace);
@@ -82,7 +82,7 @@ public class Namespaces extends NamespacesBase {
getNamespacePolicies(namespaceName);
try {
- return pulsar().getNamespaceService().getListOfDestinations(namespaceName);
+ return pulsar().getNamespaceService().getListOfTopics(namespaceName);
} catch (Exception e) {
log.error("Failed to get topics list for namespace {}", namespaceName, e);
throw new RestException(e);
@@ -263,7 +263,7 @@ public class Namespaces extends NamespacesBase {
@PUT
@Path("/{property}/{namespace}/unload")
@ApiOperation(value = "Unload namespace", notes = "Unload an active namespace from the current broker serving it. Performing this operation will let the broker"
- + "removes all producers, consumers, and connections using this namespace, and close all destinations (including"
+ + "removes all producers, consumers, and connections using this namespace, and close all topics (including"
+ "their persistent store). During that operation, the namespace is marked as tentatively unavailable until the"
+ "broker completes the unloading action. This operation requires strictly super user privileges, since it would"
+ "result in non-persistent message loss and unexpected connection closure to the clients.")
@@ -335,7 +335,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/backlogQuota")
- @ApiOperation(value = " Set a backlog quota for all the destinations on a namespace.")
+ @ApiOperation(value = " Set a backlog quota for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification"),
@@ -384,7 +384,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/persistence")
- @ApiOperation(value = "Set the persistence configuration for all the destinations on a namespace.")
+ @ApiOperation(value = "Set the persistence configuration for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification"),
@@ -409,7 +409,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/clearBacklog")
- @ApiOperation(value = "Clear backlog for all destinations on a namespace.")
+ @ApiOperation(value = "Clear backlog for all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBacklog(@PathParam("property") String property, @PathParam("namespace") String namespace,
@@ -420,7 +420,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/{bundle}/clearBacklog")
- @ApiOperation(value = "Clear backlog for all destinations on a namespace bundle.")
+ @ApiOperation(value = "Clear backlog for all topics on a namespace bundle.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBundleBacklog(@PathParam("property") String property,
@@ -432,7 +432,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/clearBacklog/{subscription}")
- @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace.")
+ @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBacklogForSubscription(@PathParam("property") String property,
@@ -444,7 +444,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/{bundle}/clearBacklog/{subscription}")
- @ApiOperation(value = "Clear backlog for a given subscription on all destinations on a namespace bundle.")
+ @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace bundle.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void clearNamespaceBundleBacklogForSubscription(@PathParam("property") String property,
@@ -457,7 +457,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/unsubscribe/{subscription}")
- @ApiOperation(value = "Unsubscribes the given subscription on all destinations on a namespace.")
+ @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public void unsubscribeNamespace(@PathParam("property") String property, @PathParam("cluster") String cluster,
@@ -482,7 +482,7 @@ public class Namespaces extends NamespacesBase {
@POST
@Path("/{property}/{namespace}/subscriptionAuthMode")
- @ApiOperation(value = " Set a subscription auth mode for all the destinations on a namespace.")
+ @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist"),
@ApiResponse(code = 409, message = "Concurrent modification") })
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
index 55bd9be..493030b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java
@@ -35,7 +35,7 @@ import javax.ws.rs.core.Response.Status;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
@@ -58,98 +58,98 @@ public class NonPersistentTopics extends PersistentTopics {
private static final Logger log = LoggerFactory.getLogger(NonPersistentTopics.class);
@GET
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Get partitioned topic metadata.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
- return getPartitionedTopicMetadata(destinationName, authoritative);
+ validateTopicName(property, namespace, encodedTopic);
+ return getPartitionedTopicMetadata(topicName, authoritative);
}
@GET
- @Path("{property}/{namespace}/{destination}/stats")
+ @Path("{property}/{namespace}/{topic}/stats")
@ApiOperation(value = "Get the stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public NonPersistentTopicStats getStats(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
- validateAdminOperationOnDestination(destinationName, authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicName(property, namespace, encodedTopic);
+ validateAdminOperationOnTopic(topicName, authoritative);
+ Topic topic = getTopicReference(topicName);
return ((NonPersistentTopic) topic).getStats();
}
@GET
- @Path("{property}/{namespace}/{destination}/internalStats")
+ @Path("{property}/{namespace}/{topic}/internalStats")
@ApiOperation(value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
- validateAdminOperationOnDestination(destinationName, authoritative);
- Topic topic = getTopicReference(destinationName);
+ validateTopicName(property, namespace, encodedTopic);
+ validateAdminOperationOnTopic(topicName, authoritative);
+ Topic topic = getTopicReference(topicName);
return topic.getInternalStats();
}
@PUT
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic already exist") })
public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+ @PathParam("topic") @Encoded String encodedTopic, int numPartitions,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
- validateAdminAccessOnProperty(destinationName.getProperty());
+ validateTopicName(property, namespace, encodedTopic);
+ validateAdminAccessOnProperty(topicName.getProperty());
if (numPartitions <= 1) {
throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 1");
}
try {
String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(),
- destinationName.getEncodedLocalName());
+ topicName.getEncodedLocalName());
byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions));
zkCreateOptimistic(path, data);
// we wait for the data to be synced in all quorums and the observers
Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS);
- log.info("[{}] Successfully created partitioned topic {}", clientAppId(), destinationName);
+ log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName);
} catch (KeeperException.NodeExistsException e) {
- log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), destinationName);
+ log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName);
throw new RestException(Status.CONFLICT, "Partitioned topic already exist");
} catch (Exception e) {
- log.error("[{}] Failed to create partitioned topic {}", clientAppId(), destinationName, e);
+ log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e);
throw new RestException(e);
}
}
@PUT
- @Path("/{property}/{namespace}/{destination}/unload")
+ @Path("/{property}/{namespace}/{topic}/unload")
@ApiOperation(value = "Unload a topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
- log.info("[{}] Unloading topic {}", clientAppId(), destinationName);
- if (destinationName.isGlobal()) {
+ validateTopicName(property, namespace, encodedTopic);
+ log.info("[{}] Unloading topic {}", clientAppId(), topicName);
+ if (topicName.isGlobal()) {
validateGlobalNamespaceOwnership(namespaceName);
}
- unloadTopic(destinationName, authoritative);
+ unloadTopic(topicName, authoritative);
}
- protected void validateAdminOperationOnDestination(DestinationName fqdn, boolean authoritative) {
- validateAdminAccessOnProperty(fqdn.getProperty());
- validateDestinationOwnership(fqdn, authoritative);
+ protected void validateAdminOperationOnTopic(TopicName topicName, boolean authoritative) {
+ validateAdminAccessOnProperty(topicName.getProperty());
+ validateTopicOwnership(topicName, authoritative);
}
- private Topic getTopicReference(DestinationName dn) {
+ private Topic getTopicReference(TopicName topicName) {
try {
- Topic topic = pulsar().getBrokerService().getTopicReference(dn.toString());
+ Topic topic = pulsar().getBrokerService().getTopicReference(topicName.toString());
checkNotNull(topic);
return topic;
} catch (Exception e) {
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 b52a5ad..0f89bfd 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
@@ -81,53 +81,53 @@ public class PersistentTopics extends PersistentTopicsBase {
}
@GET
- @Path("/{property}/{namespace}/{destination}/permissions")
- @ApiOperation(value = "Get permissions on a destination.", notes = "Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the"
- + "namespace level combined (union) with any eventual specific permission set on the destination.")
+ @Path("/{property}/{namespace}/{topic}/permissions")
+ @ApiOperation(value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the"
+ + "namespace level combined (union) with any eventual specific permission set on the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist") })
- public Map<String, Set<AuthAction>> getPermissionsOnDestination(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic) {
- validateDestinationName(property, namespace, encodedTopic);
- return internalGetPermissionsOnDestination();
+ public Map<String, Set<AuthAction>> getPermissionsOnTopic(@PathParam("property") String property,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic) {
+ validateTopicName(property, namespace, encodedTopic);
+ return internalGetPermissionsOnTopic();
}
@POST
- @Path("/{property}/{namespace}/{destination}/permissions/{role}")
+ @Path("/{property}/{namespace}/{topic}/permissions/{role}")
@ApiOperation(value = "Grant a new permission to a role on a single topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist"),
@ApiResponse(code = 409, message = "Concurrent modification") })
- public void grantPermissionsOnDestination(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ public void grantPermissionsOnTopic(@PathParam("property") String property,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("role") String role, Set<AuthAction> actions) {
- validateDestinationName(property, namespace, encodedTopic);
- internalGrantPermissionsOnDestination(role, actions);
+ validateTopicName(property, namespace, encodedTopic);
+ internalGrantPermissionsOnTopic(role, actions);
}
@DELETE
- @Path("/{property}/{namespace}/{destination}/permissions/{role}")
- @ApiOperation(value = "Revoke permissions on a destination.", notes = "Revoke permissions to a role on a single destination. If the permission was not set at the destination"
+ @Path("/{property}/{namespace}/{topic}/permissions/{role}")
+ @ApiOperation(value = "Revoke permissions on a topic.", notes = "Revoke permissions to a role on a single topic. If the permission was not set at the topic"
+ "level, but rather at the namespace level, this operation will return an error (HTTP status code 412).")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace doesn't exist"),
- @ApiResponse(code = 412, message = "Permissions are not set at the destination level") })
- public void revokePermissionsOnDestination(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @ApiResponse(code = 412, message = "Permissions are not set at the topic level") })
+ public void revokePermissionsOnTopic(@PathParam("property") String property,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("role") String role) {
- validateDestinationName(property, namespace, encodedTopic);
- internalRevokePermissionsOnDestination(role);
+ validateTopicName(property, namespace, encodedTopic);
+ internalRevokePermissionsOnTopic(role);
}
@PUT
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic already exist") })
public void createPartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, int numPartitions,
+ @PathParam("topic") @Encoded String encodedTopic, int numPartitions,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalCreatePartitionedTopic(numPartitions, authoritative);
}
@@ -143,254 +143,254 @@ public class PersistentTopics extends PersistentTopicsBase {
* @param property
* @param cluster
* @param namespace
- * @param destination
+ * @param topic
* @param numPartitions
*/
@POST
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Increment partitons of an existing partitioned topic.", notes = "It only increments partitions of existing non-global partitioned-topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 409, message = "Partitioned topic does not exist") })
public void updatePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, int numPartitions) {
- validateDestinationName(property, namespace, encodedTopic);
+ @PathParam("topic") @Encoded String encodedTopic, int numPartitions) {
+ validateTopicName(property, namespace, encodedTopic);
internalUpdatePartitionedTopic(numPartitions);
}
@GET
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Get partitioned topic metadata.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") })
public PartitionedTopicMetadata getPartitionedMetadata(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetPartitionedMetadata(authoritative);
}
@DELETE
- @Path("/{property}/{namespace}/{destination}/partitions")
+ @Path("/{property}/{namespace}/{topic}/partitions")
@ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Partitioned topic does not exist") })
public void deletePartitionedTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalDeletePartitionedTopic(authoritative);
}
@PUT
- @Path("/{property}/{namespace}/{destination}/unload")
+ @Path("/{property}/{namespace}/{topic}/unload")
@ApiOperation(value = "Unload a topic")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void unloadTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalUnloadTopic(authoritative);
}
@DELETE
- @Path("/{property}/{namespace}/{destination}")
+ @Path("/{property}/{namespace}/{topic}")
@ApiOperation(value = "Delete a topic.", notes = "The topic cannot be deleted if there's any active subscription or producer connected to the it.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist"),
@ApiResponse(code = 412, message = "Topic has active producers/subscriptions") })
public void deleteTopic(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalDeleteTopic(authoritative);
}
@GET
- @Path("/{property}/{namespace}/{destination}/subscriptions")
+ @Path("/{property}/{namespace}/{topic}/subscriptions")
@ApiOperation(value = "Get the list of persistent subscriptions for a given topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public List<String> getSubscriptions(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetSubscriptions(authoritative);
}
@GET
- @Path("{property}/{namespace}/{destination}/stats")
+ @Path("{property}/{namespace}/{topic}/stats")
@ApiOperation(value = "Get the stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicStats getStats(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetStats(authoritative);
}
@GET
- @Path("{property}/{namespace}/{destination}/internalStats")
+ @Path("{property}/{namespace}/{topic}/internalStats")
@ApiOperation(value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PersistentTopicInternalStats getInternalStats(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetInternalStats(authoritative);
}
@GET
- @Path("{property}/{namespace}/{destination}/internal-info")
+ @Path("{property}/{namespace}/{topic}/internal-info")
@ApiOperation(value = "Get the internal stats for the topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public void getManagedLedgerInfo(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) {
- validateDestinationName(property, namespace, encodedTopic);
+ @PathParam("topic") @Encoded String encodedTopic, @Suspended AsyncResponse asyncResponse) {
+ validateTopicName(property, namespace, encodedTopic);
internalGetManagedLedgerInfo(asyncResponse);
}
@GET
- @Path("{property}/{namespace}/{destination}/partitioned-stats")
+ @Path("{property}/{namespace}/{topic}/partitioned-stats")
@ApiOperation(value = "Get the stats for the partitioned topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public PartitionedTopicStats getPartitionedStats(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetPartitionedStats(authoritative);
}
@DELETE
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}")
@ApiOperation(value = "Delete a subscription.", notes = "There should not be any active consumers on the subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic does not exist"),
@ApiResponse(code = 412, message = "Subscription has active consumers") })
public void deleteSubscription(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalDeleteSubscription(subName, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip_all")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/skip_all")
@ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void skipAllMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalSkipAllMessages(subName, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/skip/{numMessages}")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}")
@ApiOperation(value = "Skip messages on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void skipMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@PathParam("numMessages") int numMessages,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalSkipMessages(subName, numMessages, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
@ApiOperation(value = "Expire messages on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void expireTopicMessages(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@PathParam("expireTimeInSeconds") int expireTimeInSeconds,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalExpireMessages(subName, expireTimeInSeconds, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/all_subscription/expireMessages/{expireTimeInSeconds}")
+ @Path("/{property}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}")
@ApiOperation(value = "Expire messages on all subscriptions of topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic or subscription does not exist") })
public void expireMessagesForAllSubscriptions(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@PathParam("expireTimeInSeconds") int expireTimeInSeconds,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalExpireMessagesForAllSubscriptions(expireTimeInSeconds, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor/{timestamp}")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/resetcursor/{timestamp}")
@ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic/Subscription does not exist") })
public void resetCursor(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@PathParam("timestamp") long timestamp,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalResetCursor(subName, timestamp, authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/resetcursor")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/resetcursor")
@ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before reseting cursor.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic/Subscription does not exist"),
@ApiResponse(code = 405, message = "Not supported for partitioned topics") })
public void resetCursorOnPosition(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative, MessageIdImpl messageId) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
internalResetCursorOnPosition(subName, authoritative, messageId);
}
@GET
- @Path("/{property}/{namespace}/{destination}/subscription/{subName}/position/{messagePosition}")
+ @Path("/{property}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}")
@ApiOperation(value = "Peek nth message on a topic subscription.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Topic, subscription or the message position does not exist") })
public Response peekNthMessage(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic, @PathParam("subName") String subName,
+ @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String subName,
@PathParam("messagePosition") int messagePosition,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalPeekNthMessage(subName, messagePosition, authoritative);
}
@GET
- @Path("{property}/{namespace}/{destination}/backlog")
+ @Path("{property}/{namespace}/{topic}/backlog")
@ApiOperation(value = "Get estimated backlog for offline topic.")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 404, message = "Namespace does not exist") })
public PersistentOfflineTopicStats getBacklog(@PathParam("property") String property,
- @PathParam("namespace") String namespace, @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalGetBacklog(authoritative);
}
@POST
- @Path("/{property}/{namespace}/{destination}/terminate")
+ @Path("/{property}/{namespace}/{topic}/terminate")
@ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more "
+ "messages to be published and will let consumer to drain existing messages in backlog")
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
@ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"),
@ApiResponse(code = 404, message = "Topic does not exist") })
public MessageId terminate(@PathParam("property") String property, @PathParam("namespace") String namespace,
- @PathParam("destination") @Encoded String encodedTopic,
+ @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative) {
- validateDestinationName(property, namespace, encodedTopic);
+ validateTopicName(property, namespace, encodedTopic);
return internalTerminate(authoritative);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
index 6baf283..8e34353 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadCalculatorImpl.java
@@ -39,7 +39,7 @@ public class SimpleLoadCalculatorImpl implements LoadCalculator {
@Override
public void recaliberateResourceUsagePerServiceUnit(LoadReport loadReport) {
// Based on each load report, calculate the average resource required by each service request characteristics
- // i.e. from the load report, we calculate that # of destinations and # of consumers are related to memory
+ // i.e. from the load report, we calculate that # of topics and # of consumers are related to memory
// usage, # of msg/s is counted toward NIC inbw and outbw and CPU, # of connections are related to the # of
// threads, etc.
// For example, based on the the current usage number of CPU on the broker and the total # of msg/s in all
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
similarity index 85%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
index 300bd84..eed3681 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/DestinationLookup.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookup.java
@@ -18,7 +18,6 @@
*/
package org.apache.pulsar.broker.lookup;
-import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
import static org.apache.pulsar.common.api.Commands.newLookupResponse;
@@ -40,6 +39,7 @@ import javax.ws.rs.container.AsyncResponse;
import javax.ws.rs.container.Suspended;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
@@ -49,14 +49,12 @@ import org.apache.pulsar.broker.web.RestException;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType;
import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
import org.apache.pulsar.common.naming.NamespaceBundle;
-import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.Codec;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.ws.rs.core.Response.Status;
import io.netty.buffer.ByteBuf;
import io.swagger.annotations.ApiResponse;
@@ -64,26 +62,26 @@ import io.swagger.annotations.ApiResponses;
@Path("/v2/destination/")
@NoSwaggerDocumentation
-public class DestinationLookup extends PulsarWebResource {
+public class TopicLookup extends PulsarWebResource {
@GET
- @Path("{destination-domain}/{property}/{cluster}/{namespace}/{dest}")
+ @Path("{topic-domain}/{property}/{cluster}/{namespace}/{topic}")
@Produces(MediaType.APPLICATION_JSON)
- public void lookupDestinationAsync(@PathParam("destination-domain") String destinationDomain,
+ public void lookupTopicAsync(@PathParam("topic-domain") String topicDomain,
@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("dest") @Encoded String dest,
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic,
@QueryParam("authoritative") @DefaultValue("false") boolean authoritative,
@Suspended AsyncResponse asyncResponse) {
- dest = Codec.decode(dest);
- DestinationDomain domain = null;
+ String topicName = Codec.decode(encodedTopic);
+ TopicDomain domain = null;
try {
- domain = DestinationDomain.getEnum(destinationDomain);
+ domain = TopicDomain.getEnum(topicDomain);
} catch (IllegalArgumentException e) {
- log.error("[{}] Invalid destination-domain {}", clientAppId(), destinationDomain, e);
+ log.error("[{}] Invalid topic-domain {}", clientAppId(), topicDomain, e);
throw new RestException(Status.METHOD_NOT_ALLOWED,
- "Unsupported destination domain " + destinationDomain);
+ "Unsupported topic domain " + topicDomain);
}
- DestinationName topic = DestinationName.get(domain.value(), property, cluster, namespace, dest);
+ TopicName topic = TopicName.get(domain.value(), property, cluster, namespace, topicName);
if (!pulsar().getBrokerService().getLookupRequestSemaphore().tryAcquire()) {
log.warn("No broker was found available for topic {}", topic);
@@ -153,23 +151,23 @@ public class DestinationLookup extends PulsarWebResource {
}
@GET
- @Path("{destination-domain}/{property}/{cluster}/{namespace}/{dest}/bundle")
+ @Path("{topic-domain}/{property}/{cluster}/{namespace}/{topic}/bundle")
@Produces(MediaType.APPLICATION_JSON)
@ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
- @ApiResponse(code = 405, message = "Invalid destination domain type") })
- public String getNamespaceBundle(@PathParam("destination-domain") String destinationDomain,
+ @ApiResponse(code = 405, message = "Invalid topic domain type") })
+ public String getNamespaceBundle(@PathParam("topic-domain") String topicDomain,
@PathParam("property") String property, @PathParam("cluster") String cluster,
- @PathParam("namespace") String namespace, @PathParam("dest") @Encoded String dest) {
- dest = Codec.decode(dest);
- DestinationDomain domain = null;
+ @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String topicName) {
+ topicName = Codec.decode(topicName);
+ TopicDomain domain = null;
try {
- domain = DestinationDomain.getEnum(destinationDomain);
+ domain = TopicDomain.getEnum(topicDomain);
} catch (IllegalArgumentException e) {
- log.error("[{}] Invalid destination-domain {}", clientAppId(), destinationDomain, e);
+ log.error("[{}] Invalid topic-domain {}", clientAppId(), topicDomain, e);
throw new RestException(Status.METHOD_NOT_ALLOWED,
- "Bundle lookup can not be done on destination domain " + destinationDomain);
+ "Bundle lookup can not be done on topic domain " + topicDomain);
}
- DestinationName topic = DestinationName.get(domain.value(), property, cluster, namespace, dest);
+ TopicName topic = TopicName.get(domain.value(), property, cluster, namespace, topicName);
validateSuperUserAccess();
try {
NamespaceBundle bundle = pulsar().getNamespaceService().getBundle(topic);
@@ -193,18 +191,18 @@ public class DestinationLookup extends PulsarWebResource {
* success(connect) response to client.
*
* @param pulsarService
- * @param fqdn
+ * @param topicName
* @param authoritative
* @param clientAppId
* @param requestId
* @return
*/
- public static CompletableFuture<ByteBuf> lookupDestinationAsync(PulsarService pulsarService, DestinationName fqdn, boolean authoritative,
+ public static CompletableFuture<ByteBuf> lookupTopicAsync(PulsarService pulsarService, TopicName topicName, boolean authoritative,
String clientAppId, AuthenticationDataSource authenticationData, long requestId) {
final CompletableFuture<ByteBuf> validationFuture = new CompletableFuture<>();
final CompletableFuture<ByteBuf> lookupfuture = new CompletableFuture<>();
- final String cluster = fqdn.getCluster();
+ final String cluster = topicName.getCluster();
// (1) validate cluster
getClusterDataIfDifferentCluster(pulsarService, cluster, clientAppId).thenAccept(differentClusterData -> {
@@ -220,19 +218,19 @@ public class DestinationLookup extends PulsarWebResource {
} else {
// (2) authorize client
try {
- checkAuthorization(pulsarService, fqdn, clientAppId, authenticationData);
+ checkAuthorization(pulsarService, topicName, clientAppId, authenticationData);
} catch (RestException authException) {
- log.warn("Failed to authorized {} on cluster {}", clientAppId, fqdn.toString());
+ log.warn("Failed to authorized {} on cluster {}", clientAppId, topicName.toString());
validationFuture.complete(
newLookupErrorResponse(ServerError.AuthorizationError, authException.getMessage(), requestId));
return;
} catch (Exception e) {
- log.warn("Unknown error while authorizing {} on cluster {}", clientAppId, fqdn.toString());
+ log.warn("Unknown error while authorizing {} on cluster {}", clientAppId, topicName.toString());
validationFuture.completeExceptionally(e);
return;
}
// (3) validate global namespace
- checkLocalOrGetPeerReplicationCluster(pulsarService, fqdn.getNamespaceObject())
+ checkLocalOrGetPeerReplicationCluster(pulsarService, topicName.getNamespaceObject())
.thenAccept(peerClusterData -> {
if (peerClusterData == null) {
// (4) all validation passed: initiate lookup
@@ -261,16 +259,16 @@ public class DestinationLookup extends PulsarWebResource {
if (validaitonFailureResponse != null) {
lookupfuture.complete(validaitonFailureResponse);
} else {
- pulsarService.getNamespaceService().getBrokerServiceUrlAsync(fqdn, authoritative)
+ pulsarService.getNamespaceService().getBrokerServiceUrlAsync(topicName, authoritative)
.thenAccept(lookupResult -> {
if (log.isDebugEnabled()) {
- log.debug("[{}] Lookup result {}", fqdn.toString(), lookupResult);
+ log.debug("[{}] Lookup result {}", topicName.toString(), lookupResult);
}
if (!lookupResult.isPresent()) {
lookupfuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady,
- "No broker was available to own " + fqdn, requestId));
+ "No broker was available to own " + topicName, requestId));
return;
}
@@ -287,10 +285,10 @@ public class DestinationLookup extends PulsarWebResource {
}
}).exceptionally(ex -> {
if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
- log.info("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+ log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
ex.getCause().getMessage());
} else {
- log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+ log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
ex.getMessage(), ex);
}
lookupfuture.complete(
@@ -301,10 +299,10 @@ public class DestinationLookup extends PulsarWebResource {
}).exceptionally(ex -> {
if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) {
- log.info("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+ log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
ex.getCause().getMessage());
} else {
- log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, fqdn.toString(),
+ log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(),
ex.getMessage(), ex);
}
@@ -325,5 +323,5 @@ public class DestinationLookup extends PulsarWebResource {
asyncResponse.resume(lookupData);
}
- private static final Logger log = LoggerFactory.getLogger(DestinationLookup.class);
+ private static final Logger log = LoggerFactory.getLogger(TopicLookup.class);
}
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 5319344..594b717 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
@@ -55,7 +55,7 @@ import org.apache.pulsar.broker.lookup.LookupResult;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -132,19 +132,19 @@ public class NamespaceService {
this.ownershipCache = new OwnershipCache(pulsar, bundleFactory);
}
- public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(DestinationName topic,
+ public CompletableFuture<Optional<LookupResult>> getBrokerServiceUrlAsync(TopicName topic,
boolean authoritative) {
return getBundleAsync(topic)
.thenCompose(bundle -> findBrokerServiceUrl(bundle, authoritative, false /* read-only */));
}
- public CompletableFuture<NamespaceBundle> getBundleAsync(DestinationName topic) {
+ public CompletableFuture<NamespaceBundle> getBundleAsync(TopicName topic) {
return bundleFactory.getBundlesAsync(topic.getNamespaceObject())
.thenApply(bundles -> bundles.findBundle(topic));
}
- public NamespaceBundle getBundle(DestinationName destination) throws Exception {
- return bundleFactory.getBundles(destination.getNamespaceObject()).findBundle(destination);
+ public NamespaceBundle getBundle(TopicName topicName) throws Exception {
+ return bundleFactory.getBundles(topicName.getNamespaceObject()).findBundle(topicName);
}
public int getBundleCount(NamespaceName namespace) throws Exception {
@@ -162,10 +162,10 @@ public class NamespaceService {
*/
public Optional<URL> getWebServiceUrl(ServiceUnitId suName, boolean authoritative, boolean isRequestHttps, boolean readOnly)
throws Exception {
- if (suName instanceof DestinationName) {
- DestinationName name = (DestinationName) suName;
+ if (suName instanceof TopicName) {
+ TopicName name = (TopicName) suName;
if (LOG.isDebugEnabled()) {
- LOG.debug("Getting web service URL of destination: {} - auth: {}", name, authoritative);
+ LOG.debug("Getting web service URL of topic: {} - auth: {}", name, authoritative);
}
return this.internalGetWebServiceUrl(getBundle(name), authoritative, isRequestHttps, readOnly).get();
}
@@ -248,7 +248,7 @@ public class NamespaceService {
if (myUrl.equals(otherUrl)) {
if (nsFullBundle != null) {
// preload heartbeat namespace
- pulsar.loadNamespaceDestinations(nsFullBundle);
+ pulsar.loadNamespaceTopics(nsFullBundle);
}
return true;
}
@@ -372,8 +372,8 @@ public class NamespaceService {
} else {
// Found owner for the namespace bundle
- // Schedule the task to pre-load destinations
- pulsar.loadNamespaceDestinations(bundle);
+ // Schedule the task to pre-load topics
+ pulsar.loadNamespaceTopics(bundle);
lookupFuture.complete(Optional.of(new LookupResult(ownerInfo)));
}
@@ -667,8 +667,8 @@ public class NamespaceService {
}
public boolean isServiceUnitOwned(ServiceUnitId suName) throws Exception {
- if (suName instanceof DestinationName) {
- return isDestinationOwned((DestinationName) suName);
+ if (suName instanceof TopicName) {
+ return isTopicOwned((TopicName) suName);
}
if (suName instanceof NamespaceName) {
@@ -682,11 +682,11 @@ public class NamespaceService {
throw new IllegalArgumentException("Invalid class of NamespaceBundle: " + suName.getClass().getName());
}
- public boolean isServiceUnitActive(DestinationName fqdn) {
+ public boolean isServiceUnitActive(TopicName topicName) {
try {
- return ownershipCache.getOwnedBundle(getBundle(fqdn)).isActive();
+ return ownershipCache.getOwnedBundle(getBundle(topicName)).isActive();
} catch (Exception e) {
- LOG.warn("Unable to find OwnedBundle for fqdn - [{}]", fqdn.toString());
+ LOG.warn("Unable to find OwnedBundle for topic - [{}]", topicName);
return false;
}
}
@@ -695,12 +695,12 @@ public class NamespaceService {
return ownershipCache.getOwnedBundle(getFullBundle(fqnn)) != null;
}
- private CompletableFuture<Boolean> isDestinationOwnedAsync(DestinationName topic) {
+ private CompletableFuture<Boolean> isTopicOwnedAsync(TopicName topic) {
return getBundleAsync(topic).thenApply(bundle -> ownershipCache.isNamespaceBundleOwned(bundle));
}
- private boolean isDestinationOwned(DestinationName fqdn) throws Exception {
- return ownershipCache.getOwnedBundle(getBundle(fqdn)) != null;
+ private boolean isTopicOwned(TopicName topicName) throws Exception {
+ return ownershipCache.getOwnedBundle(getBundle(topicName)) != null;
}
public void removeOwnedServiceUnit(NamespaceName nsName) throws Exception {
@@ -722,12 +722,12 @@ public class NamespaceService {
return bundleFactory;
}
- public ServiceUnitId getServiceUnitId(DestinationName destinationName) throws Exception {
- return getBundle(destinationName);
+ public ServiceUnitId getServiceUnitId(TopicName topicName) throws Exception {
+ return getBundle(topicName);
}
- public List<String> getListOfDestinations(NamespaceName namespaceName) throws Exception {
- List<String> destinations = Lists.newArrayList();
+ public List<String> getListOfTopics(NamespaceName namespaceName) throws Exception {
+ List<String> topics = Lists.newArrayList();
// For every topic there will be a managed ledger created.
try {
@@ -736,15 +736,15 @@ public class NamespaceService {
LOG.debug("Getting children from managed-ledgers now: {}", path);
}
- for (String destination : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
- destinations.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(destination)));
+ for (String topic : pulsar.getLocalZkCacheService().managedLedgerListCache().get(path)) {
+ topics.add(String.format("persistent://%s/%s", namespaceName, Codec.decode(topic)));
}
} catch (KeeperException.NoNodeException e) {
// NoNode means there are no persistent topics for this namespace
}
- destinations.sort(null);
- return destinations;
+ topics.sort(null);
+ return topics;
}
public Optional<NamespaceEphemeralData> getOwner(NamespaceBundle bundle) throws Exception {
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 4642a85..1919008 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
@@ -32,7 +32,7 @@ import org.apache.pulsar.client.api.ProducerBuilder;
import org.apache.pulsar.client.impl.Backoff;
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -217,28 +217,28 @@ public abstract class AbstractReplicator {
/**
* Replication can't be started on root-partitioned-topic to avoid producer startup conflict.
- *
+ *
* <pre>
* eg:
* if topic : persistent://prop/cluster/ns/my-topic is a partitioned topic with 2 partitions then
* broker explicitly creates replicator producer for: "my-topic-partition-1" and "my-topic-partition-2".
- *
- * However, if broker tries to start producer with root topic "my-topic" then client-lib internally creates individual
- * producers for "my-topic-partition-1" and "my-topic-partition-2" which creates conflict with existing
+ *
+ * However, if broker tries to start producer with root topic "my-topic" then client-lib internally creates individual
+ * producers for "my-topic-partition-1" and "my-topic-partition-2" which creates conflict with existing
* replicator producers.
* </pre>
- *
+ *
* Therefore, replicator can't be started on root-partition topic which can internally create multiple partitioned
* producers.
- *
+ *
* @param topicName
* @param brokerService
*/
- private void validatePartitionedTopic(String topicName, BrokerService brokerService) throws NamingException {
- DestinationName destination = DestinationName.get(topicName);
+ private void validatePartitionedTopic(String topic, BrokerService brokerService) throws NamingException {
+ TopicName topicName = TopicName.get(topic);
String partitionedTopicPath = path(AdminResource.PARTITIONED_TOPIC_PATH_ZNODE,
- destination.getNamespace().toString(), destination.getDomain().toString(),
- destination.getEncodedLocalName());
+ topicName.getNamespace().toString(), topicName.getDomain().toString(),
+ topicName.getEncodedLocalName());
boolean isPartitionedTopic = false;
try {
isPartitionedTopic = brokerService.pulsar().getConfigurationCache().policiesCache()
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
index 556a1f8..e7e38dd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java
@@ -27,7 +27,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
@@ -82,8 +82,8 @@ public class BacklogQuotaManager {
* Topic on which backlog has been exceeded
*/
public void handleExceededBacklogQuota(PersistentTopic persistentTopic) {
- DestinationName destination = DestinationName.get(persistentTopic.getName());
- String namespace = destination.getNamespace();
+ TopicName topicName = TopicName.get(persistentTopic.getName());
+ String namespace = topicName.getNamespace();
String policyPath = AdminResource.path(POLICIES, namespace);
BacklogQuota quota = getBacklogQuota(namespace, policyPath);
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 0c8fa43..20b1ce4 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
@@ -71,9 +71,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceExcept
import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
-import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
-import org.apache.pulsar.broker.service.persistent.PersistentReplicator;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
import org.apache.pulsar.broker.web.PulsarWebResource;
@@ -85,13 +83,12 @@ import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.configuration.FieldContext;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
-import org.apache.pulsar.common.policies.data.DispatchRate;
import org.apache.pulsar.common.policies.data.PersistencePolicies;
import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats;
import org.apache.pulsar.common.policies.data.PersistentTopicStats;
@@ -148,7 +145,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
private final EventLoopGroup workerGroup;
private final OrderedSafeExecutor topicOrderedExecutor;
// offline topic backlog cache
- private final ConcurrentOpenHashMap<DestinationName, PersistentOfflineTopicStats> offlineTopicStatCache;
+ private final ConcurrentOpenHashMap<TopicName, PersistentOfflineTopicStats> offlineTopicStatCache;
private static final ConcurrentOpenHashMap<String, ConfigField> dynamicConfigurationMap = prepareDynamicConfigurationMap();
private final ConcurrentOpenHashMap<String, Consumer> configRegisteredListeners;
@@ -428,7 +425,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
return topicFuture;
}
}
- final boolean isPersistentTopic = DestinationName.get(topic).getDomain().equals(DestinationDomain.persistent);
+ final boolean isPersistentTopic = TopicName.get(topic).getDomain().equals(TopicDomain.persistent);
return topics.computeIfAbsent(topic, (topicName) -> {
return isPersistentTopic ? this.createPersistentTopic(topicName)
: createNonPersistentTopic(topicName);
@@ -466,7 +463,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
log.info("Created topic {}", nonPersistentTopic);
long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs;
pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs);
- addTopicToStatsMaps(DestinationName.get(topic), nonPersistentTopic);
+ addTopicToStatsMaps(TopicName.get(topic), nonPersistentTopic);
topicFuture.complete(nonPersistentTopic);
});
replicationFuture.exceptionally((ex) -> {
@@ -568,8 +565,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
private void createPersistentTopic(final String topic, CompletableFuture<Topic> topicFuture) {
final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime());
- DestinationName destinationName = DestinationName.get(topic);
- if (!pulsar.getNamespaceService().isServiceUnitActive(destinationName)) {
+ TopicName topicName = TopicName.get(topic);
+ if (!pulsar.getNamespaceService().isServiceUnitActive(topicName)) {
// namespace is being unloaded
String msg = String.format("Namespace is being unloaded, cannot add topic %s", topic);
log.warn(msg);
@@ -578,9 +575,9 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
return;
}
- getManagedLedgerConfig(destinationName).thenAccept(managedLedgerConfig -> {
+ getManagedLedgerConfig(topicName).thenAccept(managedLedgerConfig -> {
// Once we have the configuration, we can proceed with the async open operation
- managedLedgerFactory.asyncOpen(destinationName.getPersistenceNamingEncoding(), managedLedgerConfig,
+ managedLedgerFactory.asyncOpen(topicName.getPersistenceNamingEncoding(), managedLedgerConfig,
new OpenLedgerCallback() {
@Override
public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
@@ -597,7 +594,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime())
- topicCreateTimeMs;
pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs);
- addTopicToStatsMaps(destinationName, persistentTopic);
+ addTopicToStatsMaps(topicName, persistentTopic);
topicFuture.complete(persistentTopic);
}).exceptionally((ex) -> {
log.warn(
@@ -635,14 +632,14 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
});
}
- public CompletableFuture<ManagedLedgerConfig> getManagedLedgerConfig(DestinationName topicName) {
+ public CompletableFuture<ManagedLedgerConfig> getManagedLedgerConfig(TopicName topicName) {
CompletableFuture<ManagedLedgerConfig> future = new CompletableFuture<>();
// Execute in background thread, since getting the policies might block if the z-node wasn't already cached
pulsar.getOrderedExecutor().submitOrdered(topicName, safeRun(() -> {
NamespaceName namespace = topicName.getNamespaceObject();
ServiceConfiguration serviceConfig = pulsar.getConfiguration();
- // Get persistence policy for this destination
+ // Get persistence policy for this topic
Optional<Policies> policies = Optional.empty();
try {
policies = pulsar
@@ -698,7 +695,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
return future;
}
- private void addTopicToStatsMaps(DestinationName topicName, Topic topic) {
+ private void addTopicToStatsMaps(TopicName topicName, Topic topic) {
try {
NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(topicName);
@@ -726,7 +723,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
if (!isEmpty(topics)) {
// add topic under new split bundles which already updated into NamespaceBundleFactory.bundleCache
topics.stream().forEach(t -> {
- addTopicToStatsMaps(DestinationName.get(t.getName()), t);
+ addTopicToStatsMaps(TopicName.get(t.getName()), t);
});
// remove old bundle from the map
synchronized (multiLayerTopicsMap) {
@@ -739,15 +736,15 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
}
}
- public PersistentOfflineTopicStats getOfflineTopicStat(DestinationName topicName) {
+ public PersistentOfflineTopicStats getOfflineTopicStat(TopicName topicName) {
return offlineTopicStatCache.get(topicName);
}
- public void cacheOfflineTopicStats(DestinationName topicName, PersistentOfflineTopicStats offlineTopicStats) {
+ public void cacheOfflineTopicStats(TopicName topicName, PersistentOfflineTopicStats offlineTopicStats) {
offlineTopicStatCache.put(topicName, offlineTopicStats);
}
- public void invalidateOfflineTopicStatCache(DestinationName topicName) {
+ public void invalidateOfflineTopicStatCache(TopicName topicName) {
PersistentOfflineTopicStats removed = offlineTopicStatCache.remove(topicName);
if (removed != null) {
log.info("Removed cached offline topic stat for {} ", topicName.getPersistenceNamingEncoding());
@@ -773,8 +770,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
pulsarStats.getDimensionMetrics(consumer);
}
- public List<Metrics> getDestinationMetrics() {
- return pulsarStats.getDestinationMetrics();
+ public List<Metrics> getTopicMetrics() {
+ return pulsarStats.getTopicMetrics();
}
public Map<String, NamespaceBundleStats> getBundleStats() {
@@ -823,8 +820,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
* @return determine if quota enforcement needs to be done for topic
*/
public boolean isBacklogExceeded(PersistentTopic topic) {
- DestinationName destination = DestinationName.get(topic.getName());
- long backlogQuotaLimitInBytes = getBacklogQuotaManager().getBacklogQuotaLimit(destination.getNamespace());
+ TopicName topicName = TopicName.get(topic.getName());
+ long backlogQuotaLimitInBytes = getBacklogQuotaManager().getBacklogQuotaLimit(topicName.getNamespace());
if (log.isDebugEnabled()) {
log.debug("[{}] - backlog quota limit = [{}]", topic.getName(), backlogQuotaLimitInBytes);
}
@@ -863,18 +860,18 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
}
void checkTopicNsOwnership(final String topic) throws RuntimeException {
- DestinationName destination = DestinationName.get(topic);
+ TopicName topicName = TopicName.get(topic);
boolean ownedByThisInstance;
try {
- ownedByThisInstance = pulsar.getNamespaceService().isServiceUnitOwned(destination);
+ ownedByThisInstance = pulsar.getNamespaceService().isServiceUnitOwned(topicName);
} catch (Exception e) {
- log.debug(String.format("Failed to check the ownership of the destination: %s", destination), e);
+ log.debug(String.format("Failed to check the ownership of the topic: %s", topicName), e);
throw new RuntimeException(new ServerMetadataException(e));
}
if (!ownedByThisInstance) {
String msg = String.format("Namespace not served by this instance. Please redo the lookup. "
- + "Request is denied: namespace=%s", destination.getNamespace());
+ + "Request is denied: namespace=%s", topicName.getNamespace());
log.warn(msg);
throw new RuntimeException(new ServiceUnitNotReadyException(msg));
}
@@ -890,7 +887,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
CompletableFuture<Integer> result = new CompletableFuture<Integer>();
List<CompletableFuture<Void>> closeFutures = Lists.newArrayList();
topics.forEach((name, topicFuture) -> {
- DestinationName topicName = DestinationName.get(name);
+ TopicName topicName = TopicName.get(name);
if (serviceUnit.includes(topicName)) {
// Topic needs to be unloaded
log.info("[{}] Unloading topic", topicName);
@@ -911,12 +908,12 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
public void removeTopicFromCache(String topic) {
try {
- DestinationName destination = DestinationName.get(topic);
- NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(destination);
+ TopicName topicName = TopicName.get(topic);
+ NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getBundle(topicName);
checkArgument(namespaceBundle instanceof NamespaceBundle);
String bundleName = namespaceBundle.toString();
- String namespaceName = destination.getNamespaceObject().toString();
+ String namespaceName = topicName.getNamespaceObject().toString();
synchronized (multiLayerTopicsMap) {
ConcurrentOpenHashMap<String, ConcurrentOpenHashMap<String, Topic>> namespaceMap = multiLayerTopicsMap
@@ -962,7 +959,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
log.info("{} updating with {}", path, data);
topics.forEach((name, topicFuture) -> {
- if (namespace.includes(DestinationName.get(name))) {
+ if (namespace.includes(TopicName.get(name))) {
// If the topic is already created, immediately apply the updated policies, otherwise once the topic is
// created it'll apply the policies update
topicFuture.thenAccept(topic -> {
@@ -1121,7 +1118,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
});
});
}
-
+
/**
* Allows a listener to listen on update of {@link ServiceConfiguration} change, so listener can take appropriate
* action if any specific config-field value has been changed.
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 0bb30cf..dc2e54e 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
@@ -44,7 +44,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.util.DateFormatter;
import org.slf4j.Logger;
@@ -481,19 +481,18 @@ public class Consumer {
}
public void checkPermissions() {
- DestinationName destination = DestinationName.get(subscription.getDestination());
+ TopicName topicName = TopicName.get(subscription.getTopicName());
if (cnx.getBrokerService().getAuthorizationService() != null) {
try {
- if (cnx.getBrokerService().getAuthorizationService().canConsume(destination, appId, authenticationData,
+ if (cnx.getBrokerService().getAuthorizationService().canConsume(topicName, appId, authenticationData,
subscription.getName())) {
return;
}
} catch (Exception e) {
- log.warn("[{}] Get unexpected error while autorizing [{}] {}", appId, subscription.getDestination(),
+ log.warn("[{}] Get unexpected error while autorizing [{}] {}", appId, subscription.getTopicName(),
e.getMessage(), e);
}
- log.info("[{}] is not allowed to consume from Destination" + " [{}] anymore", appId,
- subscription.getDestination());
+ log.info("[{}] is not allowed to consume from topic [{}] anymore", appId, subscription.getTopicName());
disconnect();
}
}
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 c0de7a6..ffe3e1c 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
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.common.api.Commands;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
import org.apache.pulsar.common.policies.data.PublisherStats;
import org.apache.pulsar.common.util.DateFormatter;
@@ -468,10 +468,10 @@ public class Producer {
}
public void checkPermissions() {
- DestinationName destination = DestinationName.get(topic.getName());
+ TopicName topicName = TopicName.get(topic.getName());
if (cnx.getBrokerService().getAuthorizationService() != null) {
try {
- if (cnx.getBrokerService().getAuthorizationService().canProduce(destination, appId,
+ if (cnx.getBrokerService().getAuthorizationService().canProduce(topicName, appId,
authenticationData)) {
return;
}
@@ -479,14 +479,14 @@ public class Producer {
log.warn("[{}] Get unexpected error while autorizing [{}] {}", appId, topic.getName(), e.getMessage(),
e);
}
- log.info("[{}] is not allowed to produce from destination [{}] anymore", appId, topic.getName());
+ log.info("[{}] is not allowed to produce on topic [{}] anymore", appId, topic.getName());
disconnect();
}
}
public void checkEncryption() {
if (topic.isEncryptionRequired() && !isEncrypted) {
- log.info("[{}] [{}] Unencrypted producer is not allowed to produce from destination [{}] anymore",
+ log.info("[{}] [{}] Unencrypted producer is not allowed to produce on topic [{}] anymore",
producerId, producerName, topic.getName());
disconnect();
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
index 835cabc..c04c7a8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
@@ -144,7 +144,7 @@ public class PulsarStats implements Closeable {
});
// end persistent topics section
topicStatsStream.endObject();
-
+
if(!tempNonPersistentTopics.isEmpty()) {
// start non-persistent topic
topicStatsStream.startObject("non-persistent");
@@ -157,9 +157,9 @@ public class PulsarStats implements Closeable {
}
});
// end non-persistent topics section
- topicStatsStream.endObject();
+ topicStatsStream.endObject();
}
-
+
// end namespace-bundle section
topicStatsStream.endObject();
});
@@ -182,7 +182,7 @@ public class PulsarStats implements Closeable {
// json end
topicStatsStream.endObject();
} catch (Exception e) {
- log.error("Unable to update destination stats", e);
+ log.error("Unable to update topic stats", e);
}
// swap metricsCollection and tempMetricsCollection
@@ -214,7 +214,7 @@ public class PulsarStats implements Closeable {
}
}
- public List<Metrics> getDestinationMetrics() {
+ public List<Metrics> getTopicMetrics() {
return metricsCollection;
}
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 4a2ecf8..935cb1f 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
@@ -21,7 +21,7 @@ package org.apache.pulsar.broker.service;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.getPartitionedTopicMetadata;
-import static org.apache.pulsar.broker.lookup.DestinationLookup.lookupDestinationAsync;
+import static org.apache.pulsar.broker.lookup.TopicLookup.lookupTopicAsync;
import static org.apache.pulsar.common.api.Commands.newLookupErrorResponse;
import static org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion.v5;
@@ -71,7 +71,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.Metadata;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.ConsumerStats;
@@ -213,7 +213,7 @@ public class ServerCnx extends PulsarHandler {
log.debug("[{}] Received Lookup from {} for {}", lookup.getTopic(), remoteAddress, requestId);
}
- DestinationName topicName = validateTopicName(lookup.getTopic(), requestId, lookup);
+ TopicName topicName = validateTopicName(lookup.getTopic(), requestId, lookup);
if (topicName == null) {
return;
}
@@ -253,7 +253,7 @@ public class ServerCnx extends PulsarHandler {
String finalOriginalPrincipal = originalPrincipal;
isProxyAuthorizedFuture.thenApply(isProxyAuthorized -> {
if (isProxyAuthorized) {
- lookupDestinationAsync(getBrokerService().pulsar(), topicName, authoritative,
+ lookupTopicAsync(getBrokerService().pulsar(), topicName, authoritative,
finalOriginalPrincipal != null ? finalOriginalPrincipal : authRole, authenticationData,
requestId).handle((lookupResponse, ex) -> {
if (ex == null) {
@@ -299,7 +299,7 @@ public class ServerCnx extends PulsarHandler {
remoteAddress, requestId);
}
- DestinationName topicName = validateTopicName(partitionMetadata.getTopic(), requestId, partitionMetadata);
+ TopicName topicName = validateTopicName(partitionMetadata.getTopic(), requestId, partitionMetadata);
if (topicName == null) {
return;
}
@@ -528,7 +528,7 @@ public class ServerCnx extends PulsarHandler {
checkArgument(state == State.Connected);
final long requestId = subscribe.getRequestId();
final long consumerId = subscribe.getConsumerId();
- DestinationName topicName = validateTopicName(subscribe.getTopic(), requestId, subscribe);
+ TopicName topicName = validateTopicName(subscribe.getTopic(), requestId, subscribe);
if (topicName == null) {
return;
}
@@ -704,7 +704,7 @@ public class ServerCnx extends PulsarHandler {
final boolean isEncrypted = cmdProducer.getEncrypted();
final Map<String, String> metadata = CommandUtils.metadataFromCommand(cmdProducer);
- DestinationName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer);
+ TopicName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer);
if (topicName == null) {
return;
}
@@ -1125,7 +1125,7 @@ public class ServerCnx extends PulsarHandler {
Topic topic = consumer.getSubscription().getTopic();
Position position = topic.getLastMessageId();
- int partitionIndex = DestinationName.getPartitionIndex(topic.getName());
+ int partitionIndex = TopicName.getPartitionIndex(topic.getName());
if (log.isDebugEnabled()) {
log.debug("[{}] [{}][{}] Get LastMessageId {} partitionIndex {}", remoteAddress,
topic.getName(), consumer.getSubscription().getName(), position, partitionIndex);
@@ -1259,9 +1259,9 @@ public class ServerCnx extends PulsarHandler {
}
}
- private DestinationName validateTopicName(String topic, long requestId, GeneratedMessageLite requestCommand) {
+ private TopicName validateTopicName(String topic, long requestId, GeneratedMessageLite requestCommand) {
try {
- return DestinationName.get(topic);
+ return TopicName.get(topic);
} catch (Throwable t) {
if (log.isDebugEnabled()) {
log.debug("[{}] Failed to parse topic name '{}'", remoteAddress, topic, t);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
index f50229a..65d613d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java
@@ -42,7 +42,7 @@ public interface Subscription {
void acknowledgeMessage(PositionImpl position, AckType ackType, Map<String,Long> properties);
- String getDestination();
+ String getTopicName();
Dispatcher getDispatcher();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
index be5374d..0b0f6d3 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java
@@ -36,7 +36,7 @@ import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.NonPersistentSubscriptionStats;
import org.apache.pulsar.utils.CopyOnWriteArrayList;
@@ -95,7 +95,7 @@ public class NonPersistentSubscription implements Subscription {
}
break;
case Failover:
- int partitionIndex = DestinationName.getPartitionIndex(topicName);
+ int partitionIndex = TopicName.getPartitionIndex(topicName);
if (partitionIndex < 0) {
// For non partition topics, assume index 0 to pick a predictable consumer
partitionIndex = 0;
@@ -149,7 +149,7 @@ public class NonPersistentSubscription implements Subscription {
}
@Override
- public String getDestination() {
+ public String getTopicName() {
return this.topicName;
}
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 5941768..a03ac6c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
@@ -62,7 +62,7 @@ import org.apache.pulsar.broker.stats.ClusterReplicationMetrics;
import org.apache.pulsar.broker.stats.NamespaceStats;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
@@ -177,7 +177,7 @@ public class NonPersistentTopic implements Topic {
try {
Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
- .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+ .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
.orElseThrow(() -> new KeeperException.NoNodeException());
isEncryptionRequired = policies.encryption_required;
} catch (Exception e) {
@@ -500,7 +500,7 @@ public class NonPersistentTopic implements Topic {
@Override
public CompletableFuture<Void> checkReplication() {
- DestinationName name = DestinationName.get(topic);
+ TopicName name = TopicName.get(topic);
if (!name.isGlobal()) {
return CompletableFuture.completedFuture(null);
}
@@ -667,7 +667,7 @@ public class NonPersistentTopic implements Topic {
return topic;
}
- public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream destStatsStream,
+ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream topicStatsStream,
ClusterReplicationMetrics replStats, String namespace) {
TopicStats topicStats = threadLocalTopicStats.get();
@@ -677,7 +677,7 @@ public class NonPersistentTopic implements Topic {
nsStats.producerCount += producers.size();
bundleStats.producerCount += producers.size();
- destStatsStream.startObject(topic);
+ topicStatsStream.startObject(topic);
producers.forEach(producer -> {
producer.updateRates();
@@ -692,18 +692,18 @@ public class NonPersistentTopic implements Topic {
});
// Creating publishers object for backward compatibility
- destStatsStream.startList("publishers");
- destStatsStream.endList();
+ topicStatsStream.startList("publishers");
+ topicStatsStream.endList();
// Start replicator stats
- destStatsStream.startObject("replication");
+ topicStatsStream.startObject("replication");
nsStats.replicatorCount += topicStats.remotePublishersStats.size();
// Close replication
- destStatsStream.endObject();
+ topicStatsStream.endObject();
// Start subscription stats
- destStatsStream.startObject("subscriptions");
+ topicStatsStream.startObject("subscriptions");
nsStats.subsCount += subscriptions.size();
subscriptions.forEach((subscriptionName, subscription) -> {
@@ -713,12 +713,12 @@ public class NonPersistentTopic implements Topic {
// Start subscription name & consumers
try {
- destStatsStream.startObject(subscriptionName);
+ topicStatsStream.startObject(subscriptionName);
Object[] consumers = subscription.getConsumers().array();
nsStats.consumerCount += consumers.length;
bundleStats.consumerCount += consumers.length;
- destStatsStream.startList("consumers");
+ topicStatsStream.startList("consumers");
subscription.getDispatcher().getMesssageDropRate().calculateRate();
for (Object consumerObj : consumers) {
@@ -731,43 +731,43 @@ public class NonPersistentTopic implements Topic {
subMsgRateRedeliver += consumerStats.msgRateRedeliver;
// Populate consumer specific stats here
- destStatsStream.startObject();
- destStatsStream.writePair("address", consumerStats.address);
- destStatsStream.writePair("consumerName", consumerStats.consumerName);
- destStatsStream.writePair("availablePermits", consumerStats.availablePermits);
- destStatsStream.writePair("connectedSince", consumerStats.connectedSince);
- destStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
- destStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
- destStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
+ topicStatsStream.startObject();
+ topicStatsStream.writePair("address", consumerStats.address);
+ topicStatsStream.writePair("consumerName", consumerStats.consumerName);
+ topicStatsStream.writePair("availablePermits", consumerStats.availablePermits);
+ topicStatsStream.writePair("connectedSince", consumerStats.connectedSince);
+ topicStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
+ topicStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
+ topicStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
if (SubType.Shared.equals(subscription.getType())) {
- destStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
- destStatsStream.writePair("blockedConsumerOnUnackedMsgs",
+ topicStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
+ topicStatsStream.writePair("blockedConsumerOnUnackedMsgs",
consumerStats.blockedConsumerOnUnackedMsgs);
}
if (consumerStats.clientVersion != null) {
- destStatsStream.writePair("clientVersion", consumerStats.clientVersion);
+ topicStatsStream.writePair("clientVersion", consumerStats.clientVersion);
}
- destStatsStream.endObject();
+ topicStatsStream.endObject();
}
// Close Consumer stats
- destStatsStream.endList();
+ topicStatsStream.endList();
// Populate subscription specific stats here
- destStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
- destStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
- destStatsStream.writePair("msgRateOut", subMsgRateOut);
- destStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
- destStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
- destStatsStream.writePair("type", subscription.getTypeString());
+ topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
+ topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
+ topicStatsStream.writePair("msgRateOut", subMsgRateOut);
+ topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
+ topicStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
+ topicStatsStream.writePair("type", subscription.getTypeString());
if (subscription.getDispatcher() != null) {
- destStatsStream.writePair("msgDropRate",
+ topicStatsStream.writePair("msgDropRate",
subscription.getDispatcher().getMesssageDropRate().getRate());
}
// Close consumers
- destStatsStream.endObject();
+ topicStatsStream.endObject();
topicStats.aggMsgRateOut += subMsgRateOut;
topicStats.aggMsgThroughputOut += subMsgThroughputOut;
@@ -779,17 +779,17 @@ public class NonPersistentTopic implements Topic {
});
// Close subscription
- destStatsStream.endObject();
+ topicStatsStream.endObject();
// Remaining dest stats.
topicStats.averageMsgSize = topicStats.aggMsgRateIn == 0.0 ? 0.0
: (topicStats.aggMsgThroughputIn / topicStats.aggMsgRateIn);
- destStatsStream.writePair("producerCount", producers.size());
- destStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
- destStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
- destStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
- destStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
- destStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
+ topicStatsStream.writePair("producerCount", producers.size());
+ topicStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
+ topicStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
+ topicStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
+ topicStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
+ topicStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
nsStats.msgRateIn += topicStats.aggMsgRateIn;
nsStats.msgRateOut += topicStats.aggMsgRateOut;
@@ -802,7 +802,7 @@ public class NonPersistentTopic implements Topic {
bundleStats.msgThroughputOut += topicStats.aggMsgThroughputOut;
// Close topic object
- destStatsStream.endObject();
+ topicStatsStream.endObject();
}
public NonPersistentTopicStats getStats() {
@@ -867,7 +867,7 @@ public class NonPersistentTopic implements Topic {
}
public boolean isActive() {
- if (DestinationName.get(topic).isGlobal()) {
+ if (TopicName.get(topic).isGlobal()) {
// No local consumers and no local producers
return !subscriptions.isEmpty() || hasLocalProducers();
}
@@ -881,7 +881,7 @@ public class NonPersistentTopic implements Topic {
} else {
if (System.nanoTime() - lastActive > TimeUnit.SECONDS.toNanos(gcIntervalInSeconds)) {
- if (DestinationName.get(topic).isGlobal()) {
+ if (TopicName.get(topic).isGlobal()) {
// For global namespace, close repl producers first.
// Once all repl producers are closed, we can delete the topic,
// provided no remote producers connected to the broker.
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
index dae7c4f..5bbd79a 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java
@@ -50,7 +50,7 @@ import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.SubscriptionStats;
import org.apache.pulsar.compaction.CompactedTopic;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
index 882c56a..55dda83 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/DispatchRateLimiter.java
@@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit;
import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
import org.apache.pulsar.broker.service.BrokerService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.DispatchRate;
import org.apache.pulsar.common.policies.data.Policies;
@@ -114,7 +114,7 @@ public class DispatchRateLimiter {
*/
private void registerLocalPoliciesListener() {
brokerService.pulsar().getConfigurationCache().policiesCache().registerListener((path, data, stat) -> {
- final NamespaceName namespace = DestinationName.get(this.topicName).getNamespaceObject();
+ final NamespaceName namespace = TopicName.get(this.topicName).getNamespaceObject();
final String cluster = brokerService.pulsar().getConfiguration().getClusterName();
final String policiesPath = path(POLICIES, namespace.toString());
if (policiesPath.equals(path)) {
@@ -141,7 +141,7 @@ public class DispatchRateLimiter {
* @return
*/
public DispatchRate getPoliciesDispatchRate() {
- final NamespaceName namespace = DestinationName.get(this.topicName).getNamespaceObject();
+ final NamespaceName namespace = TopicName.get(this.topicName).getNamespaceObject();
final String cluster = brokerService.pulsar().getConfiguration().getClusterName();
final String path = path(POLICIES, namespace.toString());
Optional<Policies> policies = Optional.empty();
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 6333f3a..e7e50b8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java
@@ -42,7 +42,7 @@ import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.service.Topic.PublishContext;
import org.apache.pulsar.common.api.Commands;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -365,7 +365,7 @@ public class MessageDeduplication {
}
private CompletableFuture<Boolean> isDeduplicationEnabled() {
- DestinationName name = DestinationName.get(topic.getName());
+ TopicName name = TopicName.get(topic.getName());
return pulsar.getConfigurationCache().policiesCache()
.getAsync(AdminResource.path(POLICIES, name.getNamespace())).thenApply(policies -> {
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 e742f14..1f0ea31 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
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.SubscriptionStats;
import org.apache.pulsar.utils.CopyOnWriteArrayList;
@@ -119,7 +119,7 @@ public class PersistentSubscription implements Subscription {
}
break;
case Failover:
- int partitionIndex = DestinationName.getPartitionIndex(topicName);
+ int partitionIndex = TopicName.getPartitionIndex(topicName);
if (partitionIndex < 0) {
// For non partition topics, assume index 0 to pick a predictable consumer
partitionIndex = 0;
@@ -234,7 +234,7 @@ public class PersistentSubscription implements Subscription {
}
@Override
- public String getDestination() {
+ public String getTopicName() {
return this.topicName;
}
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 88813ea..6e6b5d8 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -81,7 +81,7 @@ import org.apache.pulsar.client.impl.BatchMessageIdImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
@@ -235,7 +235,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
try {
Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
- .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+ .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
.orElseThrow(() -> new KeeperException.NoNodeException());
isEncryptionRequired = policies.encryption_required;
} catch (Exception e) {
@@ -381,7 +381,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
// read repl-cluster from policies to avoid restart of replicator which are in process of disconnect and close
try {
Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache()
- .get(AdminResource.path(POLICIES, DestinationName.get(topic).getNamespace()))
+ .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace()))
.orElseThrow(() -> new KeeperException.NoNodeException());
if (policies.replication_clusters != null) {
Set<String> configuredClusters = Sets.newTreeSet(policies.replication_clusters);
@@ -791,7 +791,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
}
private CompletableFuture<Void> checkPersistencePolicies() {
- DestinationName topicName = DestinationName.get(topic);
+ TopicName topicName = TopicName.get(topic);
CompletableFuture<Void> future = new CompletableFuture<>();
brokerService.getManagedLedgerConfig(topicName).thenAccept(config -> {
// update managed-ledger config and managed-cursor.markDeleteRate
@@ -807,7 +807,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
@Override
public CompletableFuture<Void> checkReplication() {
- DestinationName name = DestinationName.get(topic);
+ TopicName name = TopicName.get(topic);
if (!name.isGlobal()) {
return CompletableFuture.completedFuture(null);
}
@@ -868,7 +868,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
@Override
public void checkMessageExpiry() {
- DestinationName name = DestinationName.get(topic);
+ TopicName name = TopicName.get(topic);
Policies policies;
try {
policies = brokerService.pulsar().getConfigurationCache().policiesCache()
@@ -1021,7 +1021,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
return ledger;
}
- public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream destStatsStream,
+ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats, StatsOutputStream topicStatsStream,
ClusterReplicationMetrics replStats, String namespace) {
TopicStats topicStats = threadLocalTopicStats.get();
@@ -1031,7 +1031,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
nsStats.producerCount += producers.size();
bundleStats.producerCount += producers.size();
- destStatsStream.startObject(topic);
+ topicStatsStream.startObject(topic);
producers.forEach(producer -> {
producer.updateRates();
@@ -1046,11 +1046,11 @@ public class PersistentTopic implements Topic, AddEntryCallback {
});
// Creating publishers object for backward compatibility
- destStatsStream.startList("publishers");
- destStatsStream.endList();
+ topicStatsStream.startList("publishers");
+ topicStatsStream.endList();
// Start replicator stats
- destStatsStream.startObject("replication");
+ topicStatsStream.startObject("replication");
nsStats.replicatorCount += topicStats.remotePublishersStats.size();
replicators.forEach((cluster, replicator) -> {
// Update replicator cursor state
@@ -1072,20 +1072,20 @@ public class PersistentTopic implements Topic, AddEntryCallback {
topicStats.aggMsgThroughputOut += rStat.msgThroughputOut;
// Populate replicator specific stats here
- destStatsStream.startObject(cluster);
- destStatsStream.writePair("connected", rStat.connected);
- destStatsStream.writePair("msgRateExpired", rStat.msgRateExpired);
- destStatsStream.writePair("msgRateIn", rStat.msgRateIn);
- destStatsStream.writePair("msgRateOut", rStat.msgRateOut);
- destStatsStream.writePair("msgThroughputIn", rStat.msgThroughputIn);
- destStatsStream.writePair("msgThroughputOut", rStat.msgThroughputOut);
- destStatsStream.writePair("replicationBacklog", rStat.replicationBacklog);
- destStatsStream.writePair("replicationDelayInSeconds", rStat.replicationDelayInSeconds);
- destStatsStream.writePair("inboundConnection", rStat.inboundConnection);
- destStatsStream.writePair("inboundConnectedSince", rStat.inboundConnectedSince);
- destStatsStream.writePair("outboundConnection", rStat.outboundConnection);
- destStatsStream.writePair("outboundConnectedSince", rStat.outboundConnectedSince);
- destStatsStream.endObject();
+ topicStatsStream.startObject(cluster);
+ topicStatsStream.writePair("connected", rStat.connected);
+ topicStatsStream.writePair("msgRateExpired", rStat.msgRateExpired);
+ topicStatsStream.writePair("msgRateIn", rStat.msgRateIn);
+ topicStatsStream.writePair("msgRateOut", rStat.msgRateOut);
+ topicStatsStream.writePair("msgThroughputIn", rStat.msgThroughputIn);
+ topicStatsStream.writePair("msgThroughputOut", rStat.msgThroughputOut);
+ topicStatsStream.writePair("replicationBacklog", rStat.replicationBacklog);
+ topicStatsStream.writePair("replicationDelayInSeconds", rStat.replicationDelayInSeconds);
+ topicStatsStream.writePair("inboundConnection", rStat.inboundConnection);
+ topicStatsStream.writePair("inboundConnectedSince", rStat.inboundConnectedSince);
+ topicStatsStream.writePair("outboundConnection", rStat.outboundConnection);
+ topicStatsStream.writePair("outboundConnectedSince", rStat.outboundConnectedSince);
+ topicStatsStream.endObject();
nsStats.msgReplBacklog += rStat.replicationBacklog;
@@ -1108,10 +1108,10 @@ public class PersistentTopic implements Topic, AddEntryCallback {
});
// Close replication
- destStatsStream.endObject();
+ topicStatsStream.endObject();
// Start subscription stats
- destStatsStream.startObject("subscriptions");
+ topicStatsStream.startObject("subscriptions");
nsStats.subsCount += subscriptions.size();
subscriptions.forEach((subscriptionName, subscription) -> {
@@ -1121,12 +1121,12 @@ public class PersistentTopic implements Topic, AddEntryCallback {
// Start subscription name & consumers
try {
- destStatsStream.startObject(subscriptionName);
+ topicStatsStream.startObject(subscriptionName);
Object[] consumers = subscription.getConsumers().array();
nsStats.consumerCount += consumers.length;
bundleStats.consumerCount += consumers.length;
- destStatsStream.startList("consumers");
+ topicStatsStream.startList("consumers");
for (Object consumerObj : consumers) {
Consumer consumer = (Consumer) consumerObj;
@@ -1138,48 +1138,48 @@ public class PersistentTopic implements Topic, AddEntryCallback {
subMsgRateRedeliver += consumerStats.msgRateRedeliver;
// Populate consumer specific stats here
- destStatsStream.startObject();
- destStatsStream.writePair("address", consumerStats.address);
- destStatsStream.writePair("consumerName", consumerStats.consumerName);
- destStatsStream.writePair("availablePermits", consumerStats.availablePermits);
- destStatsStream.writePair("connectedSince", consumerStats.connectedSince);
- destStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
- destStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
- destStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
+ topicStatsStream.startObject();
+ topicStatsStream.writePair("address", consumerStats.address);
+ topicStatsStream.writePair("consumerName", consumerStats.consumerName);
+ topicStatsStream.writePair("availablePermits", consumerStats.availablePermits);
+ topicStatsStream.writePair("connectedSince", consumerStats.connectedSince);
+ topicStatsStream.writePair("msgRateOut", consumerStats.msgRateOut);
+ topicStatsStream.writePair("msgThroughputOut", consumerStats.msgThroughputOut);
+ topicStatsStream.writePair("msgRateRedeliver", consumerStats.msgRateRedeliver);
if (SubType.Shared.equals(subscription.getType())) {
- destStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
- destStatsStream.writePair("blockedConsumerOnUnackedMsgs",
+ topicStatsStream.writePair("unackedMessages", consumerStats.unackedMessages);
+ topicStatsStream.writePair("blockedConsumerOnUnackedMsgs",
consumerStats.blockedConsumerOnUnackedMsgs);
}
if (consumerStats.clientVersion != null) {
- destStatsStream.writePair("clientVersion", consumerStats.clientVersion);
+ topicStatsStream.writePair("clientVersion", consumerStats.clientVersion);
}
- destStatsStream.endObject();
+ topicStatsStream.endObject();
}
// Close Consumer stats
- destStatsStream.endList();
+ topicStatsStream.endList();
// Populate subscription specific stats here
- destStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
- destStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
- destStatsStream.writePair("msgRateOut", subMsgRateOut);
- destStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
- destStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
- destStatsStream.writePair("numberOfEntriesSinceFirstNotAckedMessage", subscription.getNumberOfEntriesSinceFirstNotAckedMessage());
- destStatsStream.writePair("totalNonContiguousDeletedMessagesRange", subscription.getTotalNonContiguousDeletedMessagesRange());
- destStatsStream.writePair("type", subscription.getTypeString());
+ topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog());
+ topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate());
+ topicStatsStream.writePair("msgRateOut", subMsgRateOut);
+ topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut);
+ topicStatsStream.writePair("msgRateRedeliver", subMsgRateRedeliver);
+ topicStatsStream.writePair("numberOfEntriesSinceFirstNotAckedMessage", subscription.getNumberOfEntriesSinceFirstNotAckedMessage());
+ topicStatsStream.writePair("totalNonContiguousDeletedMessagesRange", subscription.getTotalNonContiguousDeletedMessagesRange());
+ topicStatsStream.writePair("type", subscription.getTypeString());
if (SubType.Shared.equals(subscription.getType())) {
if(subscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) {
PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers)subscription.getDispatcher();
- destStatsStream.writePair("blockedSubscriptionOnUnackedMsgs", dispatcher.isBlockedDispatcherOnUnackedMsgs());
- destStatsStream.writePair("unackedMessages", dispatcher.getTotalUnackedMessages());
+ topicStatsStream.writePair("blockedSubscriptionOnUnackedMsgs", dispatcher.isBlockedDispatcherOnUnackedMsgs());
+ topicStatsStream.writePair("unackedMessages", dispatcher.getTotalUnackedMessages());
}
}
// Close consumers
- destStatsStream.endObject();
+ topicStatsStream.endObject();
topicStats.aggMsgRateOut += subMsgRateOut;
topicStats.aggMsgThroughputOut += subMsgThroughputOut;
@@ -1191,19 +1191,19 @@ public class PersistentTopic implements Topic, AddEntryCallback {
});
// Close subscription
- destStatsStream.endObject();
+ topicStatsStream.endObject();
// Remaining dest stats.
topicStats.averageMsgSize = topicStats.aggMsgRateIn == 0.0 ? 0.0
: (topicStats.aggMsgThroughputIn / topicStats.aggMsgRateIn);
- destStatsStream.writePair("producerCount", producers.size());
- destStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
- destStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
- destStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
- destStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
- destStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
- destStatsStream.writePair("storageSize", ledger.getEstimatedBacklogSize());
- destStatsStream.writePair("pendingAddEntriesCount", ((ManagedLedgerImpl) ledger).getPendingAddEntriesCount());
+ topicStatsStream.writePair("producerCount", producers.size());
+ topicStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize);
+ topicStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn);
+ topicStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut);
+ topicStatsStream.writePair("msgThroughputIn", topicStats.aggMsgThroughputIn);
+ topicStatsStream.writePair("msgThroughputOut", topicStats.aggMsgThroughputOut);
+ topicStatsStream.writePair("storageSize", ledger.getEstimatedBacklogSize());
+ topicStatsStream.writePair("pendingAddEntriesCount", ((ManagedLedgerImpl) ledger).getPendingAddEntriesCount());
nsStats.msgRateIn += topicStats.aggMsgRateIn;
nsStats.msgRateOut += topicStats.aggMsgRateOut;
@@ -1218,7 +1218,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
bundleStats.cacheSize += ((ManagedLedgerImpl) ledger).getCacheSize();
// Close topic object
- destStatsStream.endObject();
+ topicStatsStream.endObject();
}
public PersistentTopicStats getStats() {
@@ -1328,7 +1328,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
}
public boolean isActive() {
- if (DestinationName.get(topic).isGlobal()) {
+ if (TopicName.get(topic).isGlobal()) {
// No local consumers and no local producers
return !subscriptions.isEmpty() || hasLocalProducers();
}
@@ -1348,7 +1348,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
} else {
CompletableFuture<Void> replCloseFuture = new CompletableFuture<>();
- if (DestinationName.get(topic).isGlobal()) {
+ if (TopicName.get(topic).isGlobal()) {
// For global namespace, close repl producers first.
// Once all repl producers are closed, we can delete the topic,
// provided no remote producers connected to the broker.
@@ -1402,7 +1402,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
* marked as inactive.
*/
private boolean shouldTopicBeRetained() {
- DestinationName name = DestinationName.get(topic);
+ TopicName name = TopicName.get(topic);
try {
Optional<Policies> policies = brokerService.pulsar().getConfigurationCache().policiesCache()
.get(AdminResource.path(POLICIES, name.getNamespace()));
@@ -1448,8 +1448,8 @@ public class PersistentTopic implements Topic, AddEntryCallback {
*/
@Override
public BacklogQuota getBacklogQuota() {
- DestinationName destination = DestinationName.get(this.getName());
- String namespace = destination.getNamespace();
+ TopicName topicName = TopicName.get(this.getName());
+ String namespace = topicName.getNamespace();
String policyPath = AdminResource.path(POLICIES, namespace);
BacklogQuota backlogQuota = brokerService.getBacklogQuotaManager().getBacklogQuota(namespace, policyPath);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
index 1b67ea0..c2ffc53 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BookieClientStatsGenerator.java
@@ -23,7 +23,7 @@ import java.util.Map;
import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import com.google.common.collect.Maps;
@@ -32,7 +32,7 @@ import com.google.common.collect.Maps;
public class BookieClientStatsGenerator {
private final PulsarService pulsar;
- // map<namespace, map<destination, bookieOpsStats>>
+ // map<namespace, map<topic, bookieOpsStats>>
private Map<String, Map<String, PendingBookieOpsStats>> nsBookieClientStatsMap;
public BookieClientStatsGenerator(PulsarService pulsar) {
@@ -49,8 +49,8 @@ public class BookieClientStatsGenerator {
pulsar.getBrokerService().getTopics().forEach((name, topicFuture) -> {
PersistentTopic persistentTopic = (PersistentTopic) topicFuture.getNow(null);
if (persistentTopic != null) {
- DestinationName destinationName = DestinationName.get(persistentTopic.getName());
- put(destinationName, persistentTopic.getManagedLedger().getStats().getPendingBookieOpsStats());
+ TopicName topicName = TopicName.get(persistentTopic.getName());
+ put(topicName, persistentTopic.getManagedLedger().getStats().getPendingBookieOpsStats());
}
});
}
@@ -58,14 +58,14 @@ public class BookieClientStatsGenerator {
return nsBookieClientStatsMap;
}
- private void put(DestinationName destinationName, PendingBookieOpsStats bookieOpsStats) {
- String namespace = destinationName.getNamespace();
+ private void put(TopicName topicName, PendingBookieOpsStats bookieOpsStats) {
+ String namespace = topicName.getNamespace();
if (!nsBookieClientStatsMap.containsKey(namespace)) {
Map<String, PendingBookieOpsStats> destBookieClientStatsMap = Maps.newTreeMap();
- destBookieClientStatsMap.put(destinationName.toString(), bookieOpsStats);
+ destBookieClientStatsMap.put(topicName.toString(), bookieOpsStats);
nsBookieClientStatsMap.put(namespace, destBookieClientStatsMap);
} else {
- nsBookieClientStatsMap.get(namespace).put(destinationName.toString(), bookieOpsStats);
+ nsBookieClientStatsMap.get(namespace).put(topicName.toString(), bookieOpsStats);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
index 62ac7ce..9472a7d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/MetricsGenerator.java
@@ -52,7 +52,7 @@ public class MetricsGenerator {
metricsCollection.addAll(jvmMetrics.generate());
metricsCollection.addAll(new ManagedLedgerCacheMetrics(pulsar).generate());
metricsCollection.addAll(new ManagedLedgerMetrics(pulsar).generate());
- metricsCollection.addAll(pulsar.getBrokerService().getDestinationMetrics());
+ metricsCollection.addAll(pulsar.getBrokerService().getTopicMetrics());
metricsCollection.addAll(pulsar.getLoadManager().get().getLoadBalancingMetrics());
return metricsCollection;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
index 3716c75..f9c73e7 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java
@@ -211,7 +211,7 @@ abstract class AbstractMetrics {
Double val = map.getOrDefault(mkey, 0.0);
map.put(mkey, val + value);
}
-
+
protected void populateMaxMap(Map<String, Long> map, String mkey, long value) {
Long existingValue = map.get(mkey);
if (existingValue == null || value > existingValue) {
@@ -220,9 +220,9 @@ abstract class AbstractMetrics {
}
/**
- * Helper to manage populating destination map
+ * Helper to manage populating topics map
*
- * @param destStatsByDimensionMap
+ * @param ledgersByDimensionMap
* @param dimensionKey
* @param destStats
*/
@@ -237,14 +237,14 @@ abstract class AbstractMetrics {
}
}
- protected void populateDimensionMap(Map<Metrics, List<PersistentTopicStats>> destStatsByDimensionMap,
+ protected void populateDimensionMap(Map<Metrics, List<PersistentTopicStats>> topicsStatsByDimensionMap,
Metrics metrics, PersistentTopicStats destStats) {
- if (!destStatsByDimensionMap.containsKey(metrics)) {
+ if (!topicsStatsByDimensionMap.containsKey(metrics)) {
// create new list
- destStatsByDimensionMap.put(metrics, Lists.newArrayList(destStats));
+ topicsStatsByDimensionMap.put(metrics, Lists.newArrayList(destStats));
} else {
// add to collection
- destStatsByDimensionMap.get(metrics).add(destStats);
+ topicsStatsByDimensionMap.get(metrics).add(destStats);
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
index ea5f322..281a795 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java
@@ -36,7 +36,7 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
private Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap;
// temp map to prepare aggregation metrics
private Map<String, Double> tempAggregatedMetricsMap;
-
+
public ManagedLedgerMetrics(PulsarService pulsar) {
super(pulsar);
this.metricsCollection = Lists.newArrayList();
@@ -54,14 +54,14 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
/**
* Aggregation by namespace (not thread-safe)
- *
+ *
* @param ledgersByDimension
* @return
*/
private List<Metrics> aggregate(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimension) {
metricsCollection.clear();
-
+
for (Entry<Metrics, List<ManagedLedgerImpl>> e : ledgersByDimension.entrySet()) {
Metrics metrics = e.getKey();
List<ManagedLedgerImpl> ledgers = e.getValue();
@@ -122,17 +122,17 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
}
/**
- * Build a map of dimensions key to list of destination stats (not thread-safe)
+ * Build a map of dimensions key to list of topic stats (not thread-safe)
* <p>
- *
+ *
* @return
*/
private Map<Metrics, List<ManagedLedgerImpl>> groupLedgersByDimension() {
ledgersByDimensionMap.clear();
-
- // get the current destinations statistics from StatsBrokerFilter
- // Map : destination-name->dest-stat
+
+ // get the current topics statistics from StatsBrokerFilter
+ // Map : topic-name->dest-stat
for (Entry<String, ManagedLedgerImpl> e : getManagedLedgers().entrySet()) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
index 60031fc..bae1019 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java
@@ -128,7 +128,7 @@ public abstract class PulsarWebResource {
public AuthenticationDataHttps clientAuthData() {
return (AuthenticationDataHttps) httpRequest.getAttribute(AuthenticationFilter.AuthenticatedDataAttributeName);
}
-
+
public boolean isRequestHttps() {
return "https".equalsIgnoreCase(httpRequest.getScheme());
}
@@ -180,7 +180,7 @@ public abstract class PulsarWebResource {
if (pulsar.getConfiguration().isAuthenticationEnabled() && pulsar.getConfiguration().isAuthorizationEnabled()) {
log.debug("check admin access on property: {} - Authenticated: {} -- role: {}", property,
(isClientAuthenticated(clientAppId)), clientAppId);
-
+
PropertyAdmin propertyAdmin;
try {
@@ -242,7 +242,7 @@ public abstract class PulsarWebResource {
try {
ClusterData differentClusterData = getClusterDataIfDifferentCluster(pulsar(), cluster, clientAppId()).get();
if (differentClusterData != null) {
- URI redirect = getRedirectionUrl(differentClusterData);
+ URI redirect = getRedirectionUrl(differentClusterData);
// redirect to the cluster requested
if (log.isDebugEnabled()) {
log.debug("[{}] Redirecting the rest call to {}: cluster={}", clientAppId(), redirect, cluster);
@@ -478,19 +478,19 @@ public abstract class PulsarWebResource {
* @param cluster
* @param namespace
*/
- protected void validateDestinationOwnership(DestinationName fqdn, boolean authoritative) {
+ protected void validateTopicOwnership(TopicName topicName, boolean authoritative) {
NamespaceService nsService = pulsar().getNamespaceService();
try {
// per function name, this is trying to acquire the whole namespace ownership
- Optional<URL> webUrl = nsService.getWebServiceUrl(fqdn, authoritative, isRequestHttps(), false);
+ Optional<URL> webUrl = nsService.getWebServiceUrl(topicName, authoritative, isRequestHttps(), false);
// Ensure we get a url
if (webUrl == null || !webUrl.isPresent()) {
log.info("Unable to get web service url");
- throw new RestException(Status.PRECONDITION_FAILED, "Failed to find ownership for destination:" + fqdn);
+ throw new RestException(Status.PRECONDITION_FAILED, "Failed to find ownership for topic:" + topicName);
}
- if (!nsService.isServiceUnitOwned(fqdn)) {
+ if (!nsService.isServiceUnitOwned(topicName)) {
boolean newAuthoritative = isLeaderBroker(pulsar());
// Replace the host and port of the current request and redirect
URI redirect = UriBuilder.fromUri(uri.getRequestUri()).host(webUrl.get().getHost())
@@ -501,15 +501,15 @@ public abstract class PulsarWebResource {
}
} catch (IllegalArgumentException iae) {
// namespace format is not valid
- log.debug(String.format("Failed to find owner for destination:%s", fqdn), iae);
- throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for destination " + fqdn);
+ log.debug(String.format("Failed to find owner for topic :%s", topicName), iae);
+ throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for topic " + topicName);
} catch (IllegalStateException ise) {
- log.debug(String.format("Failed to find owner for destination:%s", fqdn), ise);
- throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for destination " + fqdn);
+ log.debug(String.format("Failed to find owner for topic:%s", topicName), ise);
+ throw new RestException(Status.PRECONDITION_FAILED, "Can't find owner for topic " + topicName);
} catch (WebApplicationException wae) {
throw wae;
} catch (Exception oe) {
- log.debug(String.format("Failed to find owner for destination:%s", fqdn), oe);
+ log.debug(String.format("Failed to find owner for topic:%s", topicName), oe);
throw new RestException(oe);
}
}
@@ -636,19 +636,19 @@ public abstract class PulsarWebResource {
return null;
}
- protected void checkConnect(DestinationName destination) throws RestException, Exception {
- checkAuthorization(pulsar(), destination, clientAppId(), clientAuthData());
+ protected void checkConnect(TopicName topicName) throws RestException, Exception {
+ checkAuthorization(pulsar(), topicName, clientAppId(), clientAuthData());
}
- protected static void checkAuthorization(PulsarService pulsarService, DestinationName destination, String role,
+ protected static void checkAuthorization(PulsarService pulsarService, TopicName topicName, String role,
AuthenticationDataSource authenticationData) throws RestException, Exception {
if (!pulsarService.getConfiguration().isAuthorizationEnabled()) {
// No enforcing of authorization policies
return;
}
// get zk policy manager
- if (!pulsarService.getBrokerService().getAuthorizationService().canLookup(destination, role, authenticationData)) {
- log.warn("[{}] Role {} is not allowed to lookup topic", destination, role);
+ if (!pulsarService.getBrokerService().getAuthorizationService().canLookup(topicName, role, authenticationData)) {
+ log.warn("[{}] Role {} is not allowed to lookup topic", topicName, role);
throw new RestException(Status.UNAUTHORIZED, "Don't have permission to connect to this namespace");
}
}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
index d62ae92..aac9aa0 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundle.java
@@ -22,7 +22,7 @@ import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.ServiceUnitId;
@@ -108,11 +108,11 @@ public class NamespaceBundle implements ServiceUnitId, Comparable<NamespaceBundl
}
@Override
- public boolean includes(DestinationName dn) {
- if (!this.nsname.equals(dn.getNamespaceObject())) {
+ public boolean includes(TopicName topicName) {
+ if (!this.nsname.equals(topicName.getNamespaceObject())) {
return false;
}
- return this.keyRange.contains(factory.getLongHashCode(dn.toString()));
+ return this.keyRange.contains(factory.getLongHashCode(topicName.toString()));
}
public String getBundleRange() {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
index 29d2b26..43fb219 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundles.java
@@ -26,7 +26,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.SortedSet;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import com.google.common.base.Objects;
@@ -86,11 +86,11 @@ public class NamespaceBundles {
}
}
- public NamespaceBundle findBundle(DestinationName dn) {
- checkArgument(this.nsname.equals(dn.getNamespaceObject()));
- long hashCode = factory.getLongHashCode(dn.toString());
+ public NamespaceBundle findBundle(TopicName topicName) {
+ checkArgument(this.nsname.equals(topicName.getNamespaceObject()));
+ long hashCode = factory.getLongHashCode(topicName.toString());
NamespaceBundle bundle = getBundle(hashCode);
- if (dn.getDomain().equals(DestinationDomain.non_persistent)) {
+ if (topicName.getDomain().equals(TopicDomain.non_persistent)) {
bundle.setHasNonPersistentTopic(true);
}
return bundle;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
index 1deb88b..e28b226 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java
@@ -107,9 +107,9 @@ public class SLAMonitoringTest {
createProperty(pulsarAdmins[BROKER_COUNT - 1]);
for (int i = 0; i < BROKER_COUNT; i++) {
- String destination = String.format("%s/%s/%s:%s", NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster",
+ String topic = String.format("%s/%s/%s:%s", NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster",
pulsarServices[i].getAdvertisedAddress(), brokerWebServicePorts[i]);
- pulsarAdmins[0].namespaces().createNamespace(destination);
+ pulsarAdmins[0].namespaces().createNamespace(topic);
}
}
@@ -177,10 +177,10 @@ public class SLAMonitoringTest {
public void testOwnershipViaAdminAfterSetup() {
for (int i = 0; i < BROKER_COUNT; i++) {
try {
- String destination = String.format("persistent://%s/%s/%s:%s/%s",
+ String topic = String.format("persistent://%s/%s/%s:%s/%s",
NamespaceService.SLA_NAMESPACE_PROPERTY, "my-cluster", pulsarServices[i].getAdvertisedAddress(),
brokerWebServicePorts[i], "my-topic");
- assertEquals(pulsarAdmins[0].lookups().lookupDestination(destination),
+ assertEquals(pulsarAdmins[0].lookups().lookupTopic(topic),
"pulsar://" + pulsarServices[i].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[i]);
} catch (Exception e) {
e.printStackTrace();
@@ -202,15 +202,15 @@ public class SLAMonitoringTest {
fail("Should be a able to close the broker index " + crashIndex + " Exception: " + e);
}
- String destination = String.format("persistent://%s/%s/%s:%s/%s", NamespaceService.SLA_NAMESPACE_PROPERTY,
+ String topic = String.format("persistent://%s/%s/%s:%s/%s", NamespaceService.SLA_NAMESPACE_PROPERTY,
"my-cluster", pulsarServices[crashIndex].getAdvertisedAddress(), brokerWebServicePorts[crashIndex], "my-topic");
- log.info("Lookup for namespace {}", destination);
+ log.info("Lookup for namespace {}", topic);
String broker = null;
try {
- broker = pulsarAdmins[BROKER_COUNT - 1].lookups().lookupDestination(destination);
- log.info("{} Namespace is owned by {}", destination, broker);
+ broker = pulsarAdmins[BROKER_COUNT - 1].lookups().lookupTopic(topic);
+ log.info("{} Namespace is owned by {}", topic, broker);
assertNotEquals(broker,
"pulsar://" + pulsarServices[crashIndex].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[crashIndex]);
} catch (PulsarAdminException e) {
@@ -230,8 +230,8 @@ public class SLAMonitoringTest {
}
try {
- broker = pulsarAdmins[0].lookups().lookupDestination(destination);
- log.info("{} Namespace is re-owned by {}", destination, broker);
+ broker = pulsarAdmins[0].lookups().lookupTopic(topic);
+ log.info("{} Namespace is re-owned by {}", topic, broker);
assertEquals(broker,
"pulsar://" + pulsarServices[crashIndex].getAdvertisedAddress() + ":" + brokerNativeBrokerPorts[crashIndex]);
} catch (PulsarAdminException e) {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index 5038116..6108ade 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -65,8 +65,8 @@ import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -164,8 +164,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
@DataProvider(name = "topicType")
public Object[][] topicTypeProvider() {
- return new Object[][] { { DestinationDomain.persistent.value() },
- { DestinationDomain.non_persistent.value() } };
+ return new Object[][] { { TopicDomain.persistent.value() },
+ { TopicDomain.non_persistent.value() } };
}
@Test
@@ -599,7 +599,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(admin.namespaces().getPolicies("prop-xyz/use/ns1"), policies);
assertEquals(admin.namespaces().getPermissions("prop-xyz/use/ns1"), policies.auth_policies.namespace_auth);
- assertEquals(admin.namespaces().getDestinations("prop-xyz/use/ns1"), Lists.newArrayList());
+ assertEquals(admin.namespaces().getTopics("prop-xyz/use/ns1"), Lists.newArrayList());
admin.namespaces().revokePermissionsOnNamespace("prop-xyz/use/ns1", "my-role");
policies.auth_policies.namespace_auth.remove("my-role");
@@ -656,7 +656,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
final String persistentTopicName = "persistent://prop-xyz/use/ns1/" + topicName;
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/" + topicName, 0);
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
Lists.newArrayList("persistent://prop-xyz/use/ns1/" + topicName));
@@ -742,8 +742,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(admin.persistentTopics().getPartitionedTopicMetadata(partitionedTopicName).partitions, 4);
// check if the virtual topic doesn't get created
- List<String> destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
- assertEquals(destinations.size(), 0);
+ List<String> topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+ assertEquals(topics.size(), 0);
assertEquals(
admin.persistentTopics().getPartitionedTopicMetadata("persistent://prop-xyz/use/ns1/ds2").partitions,
@@ -836,8 +836,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
producer = client.createProducer(partitionedTopicName);
- destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
- assertEquals(destinations.size(), 4);
+ topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+ assertEquals(topics.size(), 4);
try {
admin.persistentTopics().deletePartitionedTopic(partitionedTopicName);
@@ -876,12 +876,12 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
admin.namespaces().createNamespace("prop-xyz/use/ns1-bundles", numBundles);
// since we have 2 brokers running, we try to let both of them acquire bundle ownership
- admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds1");
- admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds2");
- admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds3");
- admin.lookups().lookupDestination("persistent://prop-xyz/use/ns1-bundles/ds4");
+ admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds1");
+ admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds2");
+ admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds3");
+ admin.lookups().lookupTopic("persistent://prop-xyz/use/ns1-bundles/ds4");
- assertEquals(admin.namespaces().getDestinations("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
+ assertEquals(admin.namespaces().getTopics("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
admin.namespaces().deleteNamespace("prop-xyz/use/ns1-bundles");
assertEquals(admin.namespaces().getNamespaces("prop-xyz", "use"), Lists.newArrayList());
@@ -889,7 +889,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
@Test
public void testNamespaceSplitBundle() throws Exception {
- // Force to create a destination
+ // Force to create a topic
final String namespace = "prop-xyz/use/ns1";
final String topicName = (new StringBuilder("persistent://")).append(namespace).append("/ds2").toString();
Producer producer = pulsarClient.createProducer(topicName);
@@ -917,7 +917,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
public void testNamespaceUnloadBundle() throws Exception {
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/ds2", 0);
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
Lists.newArrayList("persistent://prop-xyz/use/ns1/ds2"));
@@ -975,7 +975,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1-bundles"), Lists.newArrayList());
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1-bundles/ds2", 0);
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1-bundles"),
Lists.newArrayList("persistent://prop-xyz/use/ns1-bundles/ds2"));
@@ -994,7 +994,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
}
NamespaceBundle bundle = (NamespaceBundle) pulsar.getNamespaceService()
- .getBundle(DestinationName.get("persistent://prop-xyz/use/ns1-bundles/ds2"));
+ .getBundle(TopicName.get("persistent://prop-xyz/use/ns1-bundles/ds2"));
consumer.close();
producer.close();
@@ -1164,7 +1164,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
}
@Test
- public void statsOnNonExistingDestinations() throws Exception {
+ public void statsOnNonExistingTopics() throws Exception {
try {
admin.persistentTopics().getStats("persistent://prop-xyz/use/ns1/ghostTopic");
fail("The topic doesn't exist");
@@ -1384,8 +1384,8 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
conf.setSubscriptionType(SubscriptionType.Exclusive);
Consumer consumer = pulsarClient.subscribe(topicName, "my-sub", conf);
- List<String> destinations = admin.persistentTopics().getList("prop-xyz/use/ns1");
- assertEquals(destinations.size(), 4);
+ List<String> topics = admin.persistentTopics().getList("prop-xyz/use/ns1");
+ assertEquals(topics.size(), 4);
assertEquals(admin.persistentTopics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));
@@ -1427,7 +1427,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList());
String topicName = "persistent://prop-xyz/use/ns1/invalidcursorreset";
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnPersistentTopic(topicName, 0);
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"), Lists.newArrayList(topicName));
@@ -1504,7 +1504,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
@Test
public void testPersistentTopicsExpireMessages() throws Exception {
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnPersistentTopic("persistent://prop-xyz/use/ns1/ds2", 0);
assertEquals(admin.persistentTopics().getList("prop-xyz/use/ns1"),
Lists.newArrayList("persistent://prop-xyz/use/ns1/ds2"));
@@ -1617,13 +1617,13 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
@Test(dataProvider = "topicName")
public void testPulsarAdminForUriAndUrlEncoding(String topicName) throws Exception {
final String ns1 = "prop-xyz/use/ns1";
- final String dn1 = "persistent://" + ns1 + "/" + topicName;
+ final String topic1 = "persistent://" + ns1 + "/" + topicName;
final String urlEncodedTopic = Codec.encode(topicName);
final String uriEncodedTopic = urlEncodedTopic.replaceAll("\\+", "%20");
final int numOfPartitions = 4;
- admin.persistentTopics().createPartitionedTopic(dn1, numOfPartitions);
+ admin.persistentTopics().createPartitionedTopic(topic1, numOfPartitions);
// Create a consumer to get stats on this topic
- pulsarClient.subscribe(dn1, "my-subscriber-name", new ConsumerConfiguration());
+ pulsarClient.subscribe(topic1, "my-subscriber-name", new ConsumerConfiguration());
PersistentTopicsImpl persistent = (PersistentTopicsImpl) admin.persistentTopics();
Field field = PersistentTopicsImpl.class.getDeclaredField("persistentTopics");
@@ -1741,7 +1741,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
}
@Test
- public void testDestinationBundleRangeLookup() throws PulsarAdminException, PulsarServerException, Exception {
+ public void testTopicBundleRangeLookup() throws PulsarAdminException, PulsarServerException, Exception {
admin.clusters().createCluster("usw", new ClusterData());
PropertyAdmin propertyAdmin = new PropertyAdmin(Lists.newArrayList("role1", "role2"),
Sets.newHashSet("use", "usw"));
@@ -1753,7 +1753,7 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
final String topicName = "persistent://prop-xyz/use/getBundleNs/topic1";
String bundleRange = admin.lookups().getBundleRange(topicName);
assertEquals(bundleRange,
- pulsar.getNamespaceService().getBundle(DestinationName.get(topicName)).getBundleRange());
+ pulsar.getNamespaceService().getBundle(TopicName.get(topicName)).getBundleRange());
}
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
index bbbb42e..711d125 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java
@@ -54,8 +54,8 @@ import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.impl.MessageIdImpl;
-import org.apache.pulsar.common.naming.DestinationDomain;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.FailureDomain;
import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
@@ -79,8 +79,6 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStr
public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
- private static final Logger LOG = LoggerFactory.getLogger(AdminApiTest2.class);
-
private MockedPulsarService mockPulsarSetup;
@@ -111,8 +109,8 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
@DataProvider(name = "topicType")
public Object[][] topicTypeProvider() {
- return new Object[][] { { DestinationDomain.persistent.value() },
- { DestinationDomain.non_persistent.value() } };
+ return new Object[][] { { TopicDomain.persistent.value() },
+ { TopicDomain.non_persistent.value() } };
}
@DataProvider(name = "namespaceNames")
@@ -170,7 +168,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
assertEquals(admin.persistentTopics().getPartitionedTopicMetadata(partitionedTopicName).partitions,
newPartitions);
// (2) No Msg loss: verify new partitions have the same existing subscription names
- final String newPartitionTopicName = DestinationName.get(partitionedTopicName).getPartition(startPartitions + 1)
+ final String newPartitionTopicName = TopicName.get(partitionedTopicName).getPartition(startPartitions + 1)
.toString();
// (3) produce messages to all partitions including newly created partitions (RoundRobin)
@@ -212,7 +210,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
assertEquals(topicStats.partitions.keySet(), partitionSet);
for (int i = 0; i < newPartitions; i++) {
PersistentTopicStats partitionStats = topicStats.partitions
- .get(DestinationName.get(partitionedTopicName).getPartition(i).toString());
+ .get(TopicName.get(partitionedTopicName).getPartition(i).toString());
assertEquals(partitionStats.publishers.size(), 1);
assertEquals(partitionStats.subscriptions.get(subName2).consumers.size(), 1);
assertEquals(partitionStats.subscriptions.get(subName2).msgBacklog, 2, 1);
@@ -235,7 +233,7 @@ public class AdminApiTest2 extends MockedPulsarServiceBaseTest {
final String topicName = "nonPersistentTopic";
final String persistentTopicName = "non-persistent://prop-xyz/use/ns1/" + topicName;
- // Force to create a destination
+ // Force to create a topic
publishMessagesOnTopic("non-persistent://prop-xyz/use/ns1/" + topicName, 0, 0);
// create consumer and subscription
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
index dec7768..8e66aca 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java
@@ -584,8 +584,8 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
assertNotNull(allocatorStats);
Map<String, Map<String, PendingBookieOpsStats>> bookieOpsStats = brokerStats.getPendingBookieOpsStats();
assertTrue(bookieOpsStats.isEmpty());
- StreamingOutput destination = brokerStats.getDestinations2();
- assertNotNull(destination);
+ StreamingOutput topic = brokerStats.getTopics2();
+ assertNotNull(topic);
try {
brokerStats.getBrokerResourceAvailability("prop", "use", "ns2");
fail("should have failed as ModularLoadManager doesn't support it");
@@ -600,7 +600,7 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
final String property = "prop-xyz";
final String cluster = "use";
final String namespace = "ns";
- final String destination = "ds1";
+ final String topic = "ds1";
Policies policies = new Policies();
doReturn(policies).when(resourceQuotas).getNamespacePolicies(NamespaceName.get(property, cluster, namespace));
doReturn("client-id").when(resourceQuotas).clientAppId();
@@ -613,11 +613,11 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
List<String> list = persistentTopics.getList(property, cluster, namespace);
assertTrue(list.isEmpty());
- // create destination
+ // create topic
assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList());
- persistentTopics.createPartitionedTopic(property, cluster, namespace, destination, 5, false);
+ persistentTopics.createPartitionedTopic(property, cluster, namespace, topic, 5, false);
assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists
- .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, destination)));
+ .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, topic)));
CountDownLatch notificationLatch = new CountDownLatch(2);
configurationCache.policiesCache().registerListener((path, data, stat) -> {
@@ -627,19 +627,19 @@ public class AdminTest extends MockedPulsarServiceBaseTest {
// grant permission
final Set<AuthAction> actions = Sets.newHashSet(AuthAction.produce);
final String role = "test-role";
- persistentTopics.grantPermissionsOnDestination(property, cluster, namespace, destination, role, actions);
+ persistentTopics.grantPermissionsOnTopic(property, cluster, namespace, topic, role, actions);
// verify permission
- Map<String, Set<AuthAction>> permission = persistentTopics.getPermissionsOnDestination(property, cluster,
- namespace, destination);
+ Map<String, Set<AuthAction>> permission = persistentTopics.getPermissionsOnTopic(property, cluster,
+ namespace, topic);
assertEquals(permission.get(role), actions);
// remove permission
- persistentTopics.revokePermissionsOnDestination(property, cluster, namespace, destination, role);
+ persistentTopics.revokePermissionsOnTopic(property, cluster, namespace, topic, role);
// Wait for cache to be updated
notificationLatch.await();
// verify removed permission
- permission = persistentTopics.getPermissionsOnDestination(property, cluster, namespace, destination);
+ permission = persistentTopics.getPermissionsOnTopic(property, cluster, namespace, topic);
assertTrue(permission.isEmpty());
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
index 6f759d8..c6f277d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
@@ -28,7 +28,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
@@ -99,7 +99,7 @@ public class CreateSubscriptionTest extends MockedPulsarServiceBaseTest {
for (int i = 0; i < 10; i++) {
assertEquals(
- admin.persistentTopics().getSubscriptions(DestinationName.get(topic).getPartition(i).toString()),
+ admin.persistentTopics().getSubscriptions(TopicName.get(topic).getPartition(i).toString()),
Lists.newArrayList("sub-1"));
}
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
index 779466a..e001ba0 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/IncrementPartitionsTest.java
@@ -23,7 +23,7 @@ import static org.testng.Assert.assertEquals;
import org.apache.pulsar.broker.admin.AdminApiTest.MockedPulsarService;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.client.api.Consumer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.testng.annotations.AfterMethod;
@@ -87,7 +87,7 @@ public class IncrementPartitionsTest extends MockedPulsarServiceBaseTest {
assertEquals(
admin.persistentTopics()
- .getSubscriptions(DestinationName.get(partitionedTopicName).getPartition(15).toString()),
+ .getSubscriptions(TopicName.get(partitionedTopicName).getPartition(15).toString()),
Lists.newArrayList("sub-1"));
consumer.close();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
index f9ff652..16ca86c 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.web.PulsarWebResource;
import org.apache.pulsar.broker.web.RestException;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundles;
import org.apache.pulsar.common.naming.NamespaceName;
@@ -598,7 +598,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
}
NamespaceName testNs = this.testLocalNamespaces.get(1);
- DestinationName topicName = DestinationName.get(testNs.getPersistentTopicName("my-topic"));
+ TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic"));
ZkUtils.createFullPathOptimistic(mockZookKeeper, "/managed-ledgers/" + topicName.getPersistenceNamingEncoding(),
new byte[0], null, null);
@@ -640,7 +640,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
// delete the topic from ZK
mockZookKeeper.delete("/managed-ledgers/" + topicName.getPersistenceNamingEncoding(), -1);
- // ensure refreshed destination list in the cache
+ // ensure refreshed topics list in the cache
pulsar.getLocalZkCacheService().managedLedgerListCache().clearTree();
// setup ownership to localhost
doReturn(Optional.of(localWebServiceUrl)).when(nsSvc).getWebServiceUrl(testNs, false, false, false);
@@ -1047,7 +1047,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
}
@Test
- public void testValidateDestinationOwnership() throws Exception {
+ public void testValidateTopicOwnership() throws Exception {
try {
URL localWebServiceUrl = new URL(pulsar.getWebServiceAddress());
String bundledNsLocal = "test-bundled-namespace-1";
@@ -1059,7 +1059,7 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
Field ownership = NamespaceService.class.getDeclaredField("ownershipCache");
ownership.setAccessible(true);
ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
- DestinationName topicName = DestinationName.get(testNs.getPersistentTopicName("my-topic"));
+ TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic"));
PersistentTopics topics = spy(new PersistentTopics());
topics.setServletContext(new MockServletContext());
topics.setPulsar(pulsar);
@@ -1069,9 +1069,9 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
doReturn("persistent").when(topics).domain();
try {
- topics.validateDestinationName(topicName.getProperty(), topicName.getCluster(),
+ topics.validateTopicName(topicName.getProperty(), topicName.getCluster(),
topicName.getNamespacePortion(), topicName.getEncodedLocalName());
- topics.validateAdminOperationOnDestination(false);
+ topics.validateAdminOperationOnTopic(false);
} catch (RestException e) {
fail("validateAdminAccessOnProperty failed");
}
@@ -1100,17 +1100,17 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
// (1) Force topic creation and namespace being loaded
final String topicName = "persistent://" + namespace + "/my-topic";
- DestinationName destination = DestinationName.get(topicName);
+ TopicName topic = TopicName.get(topicName);
Producer producer = pulsarClient.createProducer(topicName);
producer.close();
- NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(destination);
+ NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(topic);
// (2) Delete topic
admin.persistentTopics().delete(topicName);
// (3) Delete ns
admin.namespaces().deleteNamespace(namespace);
// (4) check bundle
- NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(destination);
+ NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(topic);
assertNotEquals(bundle1.getBundleRange(), bundle2.getBundleRange());
// returns full bundle if policies not present
assertEquals("0x00000000_0xffffffff", bundle2.getBundleRange());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
index c95fbd7..4f0baf6 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java
@@ -24,7 +24,7 @@ import static org.testng.Assert.fail;
import java.util.EnumSet;
import org.apache.pulsar.broker.authorization.AuthorizationService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -63,7 +63,7 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
void simple() throws Exception {
AuthorizationService auth = pulsar.getBrokerService().getAuthorizationService();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
admin.clusters().updateCluster("c1", new ClusterData());
admin.properties().createProperty("p1", new PropertyAdmin(Lists.newArrayList("role1"), Sets.newHashSet("c1")));
@@ -71,120 +71,120 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
admin.namespaces().createNamespace("p1/c1/ns1");
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.of(AuthAction.produce));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role",
EnumSet.of(AuthAction.consume));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.allOf(AuthAction.class));
waitForChange();
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
// test for wildcard
// namespace prefix match
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my.role.*", EnumSet.of(AuthAction.produce));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
// namespace suffix match
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "*.role.my", EnumSet.of(AuthAction.consume));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
// revoke for next test
admin.namespaces().revokePermissionsOnNamespace("p1/c1/ns1", "my.role.*");
admin.namespaces().revokePermissionsOnNamespace("p1/c1/ns1", "*.role.my");
waitForChange();
- // destination prefix match
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
+ // topic prefix match
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), false);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds1", "my.*",
EnumSet.of(AuthAction.produce));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
-
- // destination suffix match
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.2", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "other.role.2", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null), false);
+
+ // topic suffix match
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), false);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds1", "*.my",
EnumSet.of(AuthAction.consume));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.my", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "1.role.my", null, null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "2.role.other", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "1.role.my", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "2.role.my", null), false);
admin.persistentTopics().revokePermissions("persistent://p1/c1/ns1/ds1", "my.*");
admin.persistentTopics().revokePermissions("persistent://p1/c1/ns1/ds1", "*.my");
@@ -194,20 +194,20 @@ public class AuthorizationTest extends MockedPulsarServiceBaseTest {
admin.namespaces().setSubscriptionAuthMode("p1/c1/ns1", SubscriptionAuthMode.Prefix);
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null), true);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null), true);
try {
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "sub1"), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "sub1"), false);
fail();
} catch (Exception e) {}
try {
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "sub2"), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "sub2"), false);
fail();
} catch (Exception e) {}
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "role1-sub1"), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2"), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1"), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role1", null, "role1-sub1"), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2"), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1"), true);
admin.namespaces().deleteNamespace("p1/c1/ns1");
admin.properties().deleteProperty("p1");
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
index 1cf6822..9ebba22 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.internal.NamespacesImpl;
import org.apache.pulsar.client.api.Authentication;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData;
import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType;
@@ -223,9 +223,9 @@ public class LoadBalancerTest {
brokerCount += entry.getValue().size();
}
assertEquals(brokerCount, BROKER_COUNT);
- DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns/test-topic");
+ TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns/test-topic");
ResourceUnit found = pulsarServices[i].getLoadManager().get()
- .getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(fqdn)).get();
+ .getLeastLoaded(pulsarServices[i].getNamespaceService().getBundle(topicName)).get();
assertTrue(found != null);
}
} catch (InterruptedException | KeeperException e) {
@@ -262,9 +262,9 @@ public class LoadBalancerTest {
int totalNamespaces = 200;
Map<String, Integer> namespaceOwner = new HashMap<>();
for (int i = 0; i < totalNamespaces; i++) {
- DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
+ TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
ResourceUnit found = pulsarServices[0].getLoadManager().get()
- .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)).get();
+ .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(topicName)).get();
if (namespaceOwner.containsKey(found.getResourceId())) {
namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
} else {
@@ -351,10 +351,10 @@ public class LoadBalancerTest {
/*
* Pre-publish load report to ZK, each broker has: - Difference memory capacity, for the first 3 brokers memory is
* bottleneck, for the 4/5th brokers CPU become bottleneck since memory is big enough - already has some bundles
- * assigned Check the distribution of new destinations is roughly consistent (with <10% variation) with the ranking
+ * assigned Check the distribution of new topics is roughly consistent (with <10% variation) with the ranking
*/
@Test
- public void testDestinationAssignmentWithExistingBundles() throws Exception {
+ public void testTopicAssignmentWithExistingBundles() throws Exception {
for (int i = 0; i < BROKER_COUNT; i++) {
ResourceQuota defaultQuota = new ResourceQuota();
defaultQuota.setMsgRateIn(20);
@@ -402,9 +402,9 @@ public class LoadBalancerTest {
int[] expectedAssignments = new int[] { 17, 34, 51, 68, 85 };
Map<String, Integer> namespaceOwner = new HashMap<>();
for (int i = 0; i < totalNamespaces; i++) {
- DestinationName fqdn = DestinationName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
+ TopicName topicName = TopicName.get("persistent://pulsar/use/primary-ns-" + i + "/test-topic");
ResourceUnit found = pulsarServices[0].getLoadManager().get()
- .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(fqdn)).get();
+ .getLeastLoaded(pulsarServices[0].getNamespaceService().getBundle(topicName)).get();
if (namespaceOwner.containsKey(found.getResourceId())) {
namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
} else {
@@ -422,7 +422,7 @@ public class LoadBalancerTest {
long expectedValue = expectedAssignments[i];
double variation = Math.abs(actualValue - expectedValue) * 100.0 / expectedValue;
- log.info("Destination assignment - {}, actual: {}, expected baseline: {}, variation: {}/%",
+ log.info("Topic assignment - {}, actual: {}, expected baseline: {}, variation: {}/%",
lookupAddresses[i], actualValue, expectedValue, String.format("%.2f", variation));
assertTrue(variation < expectedMaxVariation);
}
@@ -951,7 +951,7 @@ public class LoadBalancerTest {
Map<String, Integer> namespaceOwner = new HashMap<String, Integer>();
for (int i = 0; i < totalNamespaces; i++) {
ResourceUnit found = loadManager
- .getLeastLoaded(DestinationName.get("persistent://pulsar/use/primary-ns/topic" + i)).get();
+ .getLeastLoaded(TopicName.get("persistent://pulsar/use/primary-ns/topic" + i)).get();
if (namespaceOwner.containsKey(found.getResourceId())) {
namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
} else {
@@ -1012,7 +1012,7 @@ public class LoadBalancerTest {
Map<String, Integer> namespaceOwner = new HashMap<String, Integer>();
for (int i = 0; i < totalNamespaces; i++) {
ResourceUnit found = loadManager
- .getLeastLoaded(DestinationName.get("persistent://pulsar/use/primary-ns/topic-" + i)).get();
+ .getLeastLoaded(TopicName.get("persistent://pulsar/use/primary-ns/topic-" + i)).get();
if (namespaceOwner.containsKey(found.getResourceId())) {
namespaceOwner.put(found.getResourceId(), namespaceOwner.get(found.getResourceId()) + 1);
} else {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
similarity index 92%
rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java
rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
index 645fff9..6eefc8f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpDestinationLookupv2Test.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java
@@ -43,14 +43,14 @@ import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
-import org.apache.pulsar.broker.lookup.DestinationLookup;
+import org.apache.pulsar.broker.lookup.TopicLookup;
import org.apache.pulsar.broker.lookup.NamespaceData;
import org.apache.pulsar.broker.lookup.RedirectData;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.web.PulsarWebResource;
import org.apache.pulsar.broker.web.RestException;
-import org.apache.pulsar.common.naming.DestinationDomain;
+import org.apache.pulsar.common.naming.TopicDomain;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.zookeeper.ZooKeeperChildrenCache;
@@ -66,7 +66,7 @@ import com.google.common.collect.Lists;
*
*
*/
-public class HttpDestinationLookupv2Test {
+public class HttpTopicLookupv2Test {
private PulsarService pulsar;
private NamespaceService ns;
@@ -119,7 +119,7 @@ public class HttpDestinationLookupv2Test {
@Test
public void crossColoLookup() throws Exception {
- DestinationLookup destLookup = spy(new DestinationLookup());
+ TopicLookup destLookup = spy(new TopicLookup());
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
@@ -132,7 +132,7 @@ public class HttpDestinationLookupv2Test {
doReturn(true).when(config).isAuthorizationEnabled();
AsyncResponse asyncResponse = mock(AsyncResponse.class);
- destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
+ destLookup.lookupTopicAsync(TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
asyncResponse);
ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -149,7 +149,7 @@ public class HttpDestinationLookupv2Test {
BrokerService brokerService = pulsar.getBrokerService();
doReturn(new Semaphore(0)).when(brokerService).getLookupRequestSemaphore();
- DestinationLookup destLookup = spy(new DestinationLookup());
+ TopicLookup destLookup = spy(new TopicLookup());
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
@@ -162,7 +162,7 @@ public class HttpDestinationLookupv2Test {
doReturn(true).when(config).isAuthorizationEnabled();
AsyncResponse asyncResponse1 = mock(AsyncResponse.class);
- destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
+ destLookup.lookupTopicAsync(TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic1", false,
asyncResponse1);
ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -187,7 +187,7 @@ public class HttpDestinationLookupv2Test {
doReturn(Optional.of(policies2)).when(policiesCache)
.get(AdminResource.path(POLICIES, property, cluster, ns2));
- DestinationLookup destLookup = spy(new DestinationLookup());
+ TopicLookup destLookup = spy(new TopicLookup());
doReturn(false).when(destLookup).isRequestHttps();
destLookup.setPulsar(pulsar);
doReturn("null").when(destLookup).clientAppId();
@@ -198,7 +198,7 @@ public class HttpDestinationLookupv2Test {
doReturn(false).when(config).isAuthorizationEnabled();
AsyncResponse asyncResponse = mock(AsyncResponse.class);
- destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), property, cluster, ns1, "empty-cluster",
+ destLookup.lookupTopicAsync(TopicDomain.persistent.value(), property, cluster, ns1, "empty-cluster",
false, asyncResponse);
ArgumentCaptor<Throwable> arg = ArgumentCaptor.forClass(Throwable.class);
@@ -206,7 +206,7 @@ public class HttpDestinationLookupv2Test {
assertEquals(arg.getValue().getClass(), RestException.class);
AsyncResponse asyncResponse2 = mock(AsyncResponse.class);
- destLookup.lookupDestinationAsync(DestinationDomain.persistent.value(), property, cluster, ns2,
+ destLookup.lookupTopicAsync(TopicDomain.persistent.value(), property, cluster, ns2,
"invalid-localCluster", false, asyncResponse2);
ArgumentCaptor<Throwable> arg2 = ArgumentCaptor.forClass(Throwable.class);
verify(asyncResponse2).resume(arg2.capture());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
index f20a877..659361c 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java
@@ -56,7 +56,7 @@ import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.ConsumerConfiguration;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -108,9 +108,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
NamespaceService namespaceService = pulsar.getNamespaceService();
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
- NamespaceBundle originalBundle = bundles.findBundle(dn);
+ NamespaceBundle originalBundle = bundles.findBundle(topicName);
// Split bundle and take ownership of split bundles
CompletableFuture<Void> result = namespaceService.splitAndOwnBundle(originalBundle, false);
@@ -178,15 +178,15 @@ public class NamespaceServiceTest extends BrokerTestBase {
NamespaceService namespaceService = pulsar.getNamespaceService();
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
- NamespaceBundle originalBundle = bundles.findBundle(dn);
+ NamespaceBundle originalBundle = bundles.findBundle(topicName);
- PersistentTopic topic = new PersistentTopic(dn.toString(), ledger, pulsar.getBrokerService());
+ PersistentTopic topic = new PersistentTopic(topicName.toString(), ledger, pulsar.getBrokerService());
Method method = pulsar.getBrokerService().getClass().getDeclaredMethod("addTopicToStatsMaps",
- DestinationName.class, Topic.class);
+ TopicName.class, Topic.class);
method.setAccessible(true);
- method.invoke(pulsar.getBrokerService(), dn, topic);
+ method.invoke(pulsar.getBrokerService(), topicName, topic);
String nspace = originalBundle.getNamespaceObject().toString();
List<Topic> list = this.pulsar.getBrokerService().getAllTopicsFromNamespaceBundle(nspace,
originalBundle.toString());
@@ -210,7 +210,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
}
// status-map should be updated with new split bundles
- NamespaceBundle splitBundle = pulsar.getNamespaceService().getBundle(dn);
+ NamespaceBundle splitBundle = pulsar.getNamespaceService().getBundle(topicName);
assertTrue(!CollectionUtils.isEmpty(
this.pulsar.getBrokerService().getAllTopicsFromNamespaceBundle(nspace, splitBundle.toString())));
@@ -231,9 +231,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
NamespaceService namespaceService = pulsar.getNamespaceService();
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
- NamespaceBundle originalBundle = bundles.findBundle(dn);
+ NamespaceBundle originalBundle = bundles.findBundle(topicName);
assertFalse(namespaceService.isNamespaceBundleDisabled(originalBundle));
@@ -283,7 +283,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
return result;
}
}).when(spyTopic).close();
- NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+ NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
try {
pulsar.getNamespaceService().unloadNamespaceBundle(bundle);
} catch (Exception e) {
@@ -322,7 +322,7 @@ public class NamespaceServiceTest extends BrokerTestBase {
return new CompletableFuture<Void>();
}
}).when(spyTopic).close();
- NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+ NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
// try to unload bundle whose topic will be stuck
pulsar.getNamespaceService().unloadNamespaceBundle(bundle, 1, TimeUnit.SECONDS);
@@ -381,9 +381,9 @@ public class NamespaceServiceTest extends BrokerTestBase {
ownership.set(pulsar.getNamespaceService(), MockOwnershipCache);
NamespaceService namespaceService = pulsar.getNamespaceService();
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = namespaceService.getNamespaceBundleFactory().getBundles(nsname);
- NamespaceBundle originalBundle = bundles.findBundle(dn);
+ NamespaceBundle originalBundle = bundles.findBundle(topicName);
// Split bundle and take ownership of split bundles
CompletableFuture<Void> result = namespaceService.splitAndOwnBundle(originalBundle, false);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
index f5711eb..d688b7f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java
@@ -150,12 +150,12 @@ public class BrokerBkEnsemblesTests {
admin.namespaces().createNamespace(ns1);
- final String dn1 = "persistent://" + ns1 + "/my-topic";
+ final String topic1 = "persistent://" + ns1 + "/my-topic";
// (1) create topic
// publish and ack messages so, cursor can create cursor-ledger and update metadata
- Consumer consumer = client.subscribe(dn1, "my-subscriber-name");
- Producer producer = client.createProducer(dn1);
+ Consumer consumer = client.subscribe(topic1, "my-subscriber-name");
+ Producer producer = client.createProducer(topic1);
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
@@ -166,7 +166,7 @@ public class BrokerBkEnsemblesTests {
consumer.acknowledge(msg);
}
- PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+ PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
ManagedCursorImpl cursor = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next();
retryStrategically((test) -> cursor.getState().equals("Open"), 5, 100);
@@ -178,7 +178,7 @@ public class BrokerBkEnsemblesTests {
// (3) remove topic and managed-ledger from broker which means topic is not closed gracefully
consumer.close();
producer.close();
- pulsar.getBrokerService().removeTopicFromCache(dn1);
+ pulsar.getBrokerService().removeTopicFromCache(topic1);
ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory();
Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers");
field.setAccessible(true);
@@ -189,8 +189,8 @@ public class BrokerBkEnsemblesTests {
// (4) Recreate topic
// publish and ack messages so, cursor can create cursor-ledger and update metadata
- consumer = client.subscribe(dn1, "my-subscriber-name");
- producer = client.createProducer(dn1);
+ consumer = client.subscribe(topic1, "my-subscriber-name");
+ producer = client.createProducer(topic1);
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
@@ -201,7 +201,7 @@ public class BrokerBkEnsemblesTests {
}
// (5) Broker should create new cursor-ledger and remove old cursor-ledger
- topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+ topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
final ManagedCursorImpl cursor1 = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next();
retryStrategically((test) -> cursor1.getState().equals("Open"), 5, 100);
long newCursorLedgerId = cursor1.getCursorLedger();
@@ -244,14 +244,14 @@ public class BrokerBkEnsemblesTests {
admin.namespaces().createNamespace(ns1);
- final String dn1 = "persistent://" + ns1 + "/my-topic";
+ final String topic1 = "persistent://" + ns1 + "/my-topic";
// Create subscription
ConsumerConfiguration consumerConfig = new ConsumerConfiguration();
consumerConfig.setReceiverQueueSize(5);
- Consumer consumer = client.subscribe(dn1, "my-subscriber-name", consumerConfig);
+ Consumer consumer = client.subscribe(topic1, "my-subscriber-name", consumerConfig);
- PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(dn1).get();
+ PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic1).get();
ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger();
ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next();
Field configField = ManagedCursorImpl.class.getDeclaredField("config");
@@ -267,7 +267,7 @@ public class BrokerBkEnsemblesTests {
BookKeeper bookKeeper = (BookKeeper) bookKeeperField.get(ml);
// (1) publish messages in 5 data-ledgers each with 20 entries under managed-ledger
- Producer producer = client.createProducer(dn1);
+ Producer producer = client.createProducer(topic1);
for (int i = 0; i < totalMessages; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
@@ -294,7 +294,7 @@ public class BrokerBkEnsemblesTests {
// clean managed-ledger and recreate topic to clean any data from the cache
producer.close();
- pulsar.getBrokerService().removeTopicFromCache(dn1);
+ pulsar.getBrokerService().removeTopicFromCache(topic1);
ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory();
Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers");
field.setAccessible(true);
@@ -306,7 +306,7 @@ public class BrokerBkEnsemblesTests {
// (3) consumer will fail to consume any message as first data-ledger is non-recoverable
Message msg = null;
// start consuming message
- consumer = client.subscribe(dn1, "my-subscriber-name");
+ consumer = client.subscribe(topic1, "my-subscriber-name");
msg = consumer.receive(1, TimeUnit.SECONDS);
Assert.assertNull(msg);
consumer.close();
@@ -317,7 +317,7 @@ public class BrokerBkEnsemblesTests {
retryStrategically((test) -> config.isAutoSkipNonRecoverableData(), 5, 100);
// (5) consumer will be able to consume 20 messages from last non-deleted ledger
- consumer = client.subscribe(dn1, "my-subscriber-name");
+ consumer = client.subscribe(topic1, "my-subscriber-name");
for (int i = 0; i < entriesPerLedger; i++) {
msg = consumer.receive(5, TimeUnit.SECONDS);
System.out.println(i);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
index 1c2b9b6..f0c28ab 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
@@ -63,7 +63,7 @@ import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.impl.auth.AuthenticationTls;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.BundlesData;
import org.apache.pulsar.common.policies.data.LocalPolicies;
@@ -114,7 +114,7 @@ public class BrokerServiceTest extends BrokerTestBase {
});
latch1.await();
- admin.lookups().lookupDestination(topic);
+ admin.lookups().lookupTopic(topic);
final CountDownLatch latch2 = new CountDownLatch(1);
service.getTopic(topic).thenAccept(t -> {
@@ -371,15 +371,15 @@ public class BrokerServiceTest extends BrokerTestBase {
}
rolloverPerIntervalStats();
- JsonObject destinationStats = brokerStatsClient.getDestinations();
- assertEquals(destinationStats.size(), 2, destinationStats.toString());
+ JsonObject topicStats = brokerStatsClient.getTopics();
+ assertEquals(topicStats.size(), 2, topicStats.toString());
for (String ns : nsList) {
- JsonObject nsObject = destinationStats.getAsJsonObject(ns);
- List<String> topicList = admin.namespaces().getDestinations(ns);
+ JsonObject nsObject = topicStats.getAsJsonObject(ns);
+ List<String> topicList = admin.namespaces().getTopics(ns);
for (String topic : topicList) {
NamespaceBundle bundle = (NamespaceBundle) pulsar.getNamespaceService()
- .getBundle(DestinationName.get(topic));
+ .getBundle(TopicName.get(topic));
JsonObject bundleObject = nsObject.getAsJsonObject(bundle.getBundleRange());
JsonObject topicObject = bundleObject.getAsJsonObject("persistent");
AtomicBoolean topicPresent = new AtomicBoolean();
@@ -396,8 +396,8 @@ public class BrokerServiceTest extends BrokerTestBase {
producer.close();
}
for (String ns : nsList) {
- List<String> destinations = admin.namespaces().getDestinations(ns);
- for (String dest : destinations) {
+ List<String> topics = admin.namespaces().getTopics(ns);
+ for (String dest : topics) {
admin.persistentTopics().delete(dest);
}
admin.namespaces().deleteNamespace(ns);
@@ -761,12 +761,12 @@ public class BrokerServiceTest extends BrokerTestBase {
// own namespace bundle
final String topicName = "persistent://" + namespace + "/my-topic";
- DestinationName destination = DestinationName.get(topicName);
+ TopicName topic = TopicName.get(topicName);
Producer producer = pulsarClient.createProducer(topicName);
producer.close();
// disable namespace-bundle
- NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(destination);
+ NamespaceBundle bundle = pulsar.getNamespaceService().getBundle(topic);
pulsar.getNamespaceService().getOwnershipCache().updateBundleState(bundle, false);
// try to create topic which should fail as bundle is disable
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
index 2426997..e25581f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java
@@ -72,7 +72,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream;
@@ -175,7 +175,7 @@ public class PersistentDispatcherFailoverConsumerTest {
NamespaceService nsSvc = mock(NamespaceService.class);
doReturn(nsSvc).when(pulsar).getNamespaceService();
doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
- doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+ doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
setupMLAsyncCallbackMocks();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
index 7bc3141..fa4da0b 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
@@ -47,7 +47,7 @@ import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
@@ -326,7 +326,7 @@ public class PersistentFailoverE2ETest extends BrokerTestBase {
int numPartitions = 4;
final String topicName = "persistent://prop/use/ns-abc/failover-topic2";
- final DestinationName destName = DestinationName.get(topicName);
+ final TopicName destName = TopicName.get(topicName);
final String subName = "sub1";
final int numMsgs = 100;
Set<String> uniqueMessages = new HashSet<>();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
index d694072..eda685a 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java
@@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.ServerCnx;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.common.api.proto.PulsarApi;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
import org.apache.zookeeper.ZooKeeper;
@@ -101,7 +101,7 @@ public class PersistentTopicConcurrentTest extends MockedBookKeeperTestCase {
NamespaceService nsSvc = mock(NamespaceService.class);
doReturn(nsSvc).when(pulsar).getNamespaceService();
doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
- doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+ doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
final List<Position> addedEntries = Lists.newArrayList();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
index 3ce76d1..ad46179 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
@@ -74,7 +74,7 @@ import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet;
@@ -548,20 +548,20 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
@Test(enabled = false)
public void testUnloadNamespace() throws Exception {
- String topicName = "persistent://prop/use/ns-abc/topic-9";
- DestinationName destinationName = DestinationName.get(topicName);
- pulsarClient.createProducer(topicName);
+ String topic = "persistent://prop/use/ns-abc/topic-9";
+ TopicName topicName = TopicName.get(topic);
+ pulsarClient.createProducer(topic);
pulsarClient.close();
- assertTrue(pulsar.getBrokerService().getTopicReference(topicName) != null);
+ assertTrue(pulsar.getBrokerService().getTopicReference(topic) != null);
assertTrue(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers()
- .containsKey(destinationName.getPersistenceNamingEncoding()));
+ .containsKey(topicName.getPersistenceNamingEncoding()));
admin.namespaces().unload("prop/use/ns-abc");
int i = 0;
for (i = 0; i < 30; i++) {
- if (pulsar.getBrokerService().getTopicReference(topicName) == null) {
+ if (pulsar.getBrokerService().getTopicReference(topic) == null) {
break;
}
Thread.sleep(1000);
@@ -572,7 +572,7 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
// ML should have been closed as well
assertFalse(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers()
- .containsKey(destinationName.getPersistenceNamingEncoding()));
+ .containsKey(topicName.getPersistenceNamingEncoding()));
}
@Test
@@ -1128,7 +1128,7 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
// sleep 1 sec to caclulate metrics per second
Thread.sleep(1000);
brokerService.updateRates();
- List<Metrics> metrics = brokerService.getDestinationMetrics();
+ List<Metrics> metrics = brokerService.getTopicMetrics();
for (int i = 0; i < metrics.size(); i++) {
if (metrics.get(i).getDimension("namespace").equalsIgnoreCase(namespace)) {
metric = metrics.get(i);
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 b088587..b19b053 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
@@ -93,7 +93,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
@@ -170,7 +170,7 @@ public class PersistentTopicTest {
NamespaceService nsSvc = mock(NamespaceService.class);
doReturn(nsSvc).when(pulsar).getNamespaceService();
doReturn(true).when(nsSvc).isServiceUnitOwned(any(NamespaceBundle.class));
- doReturn(true).when(nsSvc).isServiceUnitActive(any(DestinationName.class));
+ doReturn(true).when(nsSvc).isServiceUnitActive(any(TopicName.class));
setupMLAsyncCallbackMocks();
}
@@ -1117,7 +1117,7 @@ public class PersistentTopicTest {
// step-2 now, policies doesn't have removed replication cluster so, it should not invoke "startProducer" of the
// replicator
when(pulsar.getConfigurationCache().policiesCache()
- .get(AdminResource.path(POLICIES, DestinationName.get(globalTopicName).getNamespace())))
+ .get(AdminResource.path(POLICIES, TopicName.get(globalTopicName).getNamespace())))
.thenReturn(Optional.of(new Policies()));
// try to start replicator again
topic.startReplProducers();
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 5fd74a9..1964db9 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
@@ -63,7 +63,7 @@ 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.api.Commands;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
@@ -124,10 +124,10 @@ public class ReplicatorTest extends ReplicatorTestBase {
log.info("--- Starting ReplicatorTest::testConfigChange ---");
// This test is to verify that the config change on global namespace is successfully applied in broker during
// runtime.
- // Run a set of producer tasks to create the destinations
+ // Run a set of producer tasks to create the topics
List<Future<Void>> results = Lists.newArrayList();
for (int i = 0; i < 10; i++) {
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/topic-%d", i));
results.add(executor.submit(new Callable<Void>() {
@@ -212,7 +212,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
final String namespace = "pulsar/global/concurrent";
admin1.namespaces().createNamespace(namespace);
admin1.namespaces().setNamespaceReplicationClusters(namespace, Lists.newArrayList("r1", "r2"));
- final DestinationName dest = DestinationName.get(String.format("persistent://" + namespace + "/topic-%d", 0));
+ final TopicName dest = TopicName.get(String.format("persistent://" + namespace + "/topic-%d", 0));
ClientConfiguration conf = new ClientConfiguration();
conf.setStatsInterval(0, TimeUnit.SECONDS);
Producer producer = PulsarClient.create(url1.toString(), conf).createProducer(dest.toString());
@@ -288,11 +288,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
// This test is to verify that the config change on global namespace is successfully applied in broker during
// runtime.
- // Run a set of producer tasks to create the destinations
+ // Run a set of producer tasks to create the topics
SortedSet<String> testDests = new TreeSet<String>();
List<Future<Void>> results = Lists.newArrayList();
for (int i = 0; i < 3; i++) {
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/repltopic-%d", i));
testDests.add(dest.toString());
@@ -389,11 +389,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
// This test is to verify that the config change on global namespace is successfully applied in broker during
// runtime.
- // Run a set of producer tasks to create the destinations
+ // Run a set of producer tasks to create the topics
SortedSet<String> testDests = new TreeSet<String>();
List<Future<Void>> results = Lists.newArrayList();
for (int i = 0; i < 10; i++) {
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/repltopic-%d", i));
testDests.add(dest.toString());
@@ -473,7 +473,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
try {
// 1. Create a consumer using the reserved consumer id prefix "pulsar.repl."
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/res-cons-id"));
// Create another consumer using replication prefix as sub id
@@ -491,7 +491,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
SortedSet<String> testDests = new TreeSet<String>();
- final DestinationName dest = DestinationName.get("persistent://pulsar/global/ns/peekAndSeekTopic");
+ final TopicName dest = TopicName.get("persistent://pulsar/global/ns/peekAndSeekTopic");
testDests.add(dest.toString());
MessageProducer producer1 = new MessageProducer(url1, dest);
@@ -515,7 +515,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
// This test is to verify that reset cursor fails on global topic
SortedSet<String> testDests = new TreeSet<String>();
- final DestinationName dest = DestinationName.get("persistent://pulsar/global/ns/clearBacklogTopic");
+ final TopicName dest = TopicName.get("persistent://pulsar/global/ns/clearBacklogTopic");
testDests.add(dest.toString());
MessageProducer producer1 = new MessageProducer(url1, dest);
@@ -545,7 +545,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
SortedSet<String> testDests = new TreeSet<String>();
List<Future<Void>> results = Lists.newArrayList();
for (int i = 0; i < 1; i++) {
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/resetrepltopic-%d", i));
testDests.add(dest.toString());
@@ -587,11 +587,11 @@ public class ReplicatorTest extends ReplicatorTestBase {
log.info("--- Starting ReplicatorTest::testReplicationForBatchMessages ---");
- // Run a set of producer tasks to create the destinations
+ // Run a set of producer tasks to create the topics
SortedSet<String> testDests = new TreeSet<String>();
List<Future<Void>> results = Lists.newArrayList();
for (int i = 0; i < 3; i++) {
- final DestinationName dest = DestinationName
+ final TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns/repltopicbatch-%d", i));
testDests.add(dest.toString());
@@ -660,7 +660,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
public void testDeleteReplicatorFailure() throws Exception {
log.info("--- Starting ReplicatorTest::testDeleteReplicatorFailure ---");
final String topicName = "persistent://pulsar/global/ns/repltopicbatch";
- final DestinationName dest = DestinationName.get(topicName);
+ final TopicName dest = TopicName.get(topicName);
MessageProducer producer1 = new MessageProducer(url1, dest);
PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName);
final String replicatorClusterName = topic.getReplicators().keys().get(0);
@@ -699,7 +699,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
public void testReplicatorProducerClosing() throws Exception {
log.info("--- Starting ReplicatorTest::testDeleteReplicatorFailure ---");
final String topicName = "persistent://pulsar/global/ns/repltopicbatch";
- final DestinationName dest = DestinationName.get(topicName);
+ final TopicName dest = TopicName.get(topicName);
MessageProducer producer1 = new MessageProducer(url1, dest);
PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName);
final String replicatorClusterName = topic.getReplicators().keys().get(0);
@@ -735,7 +735,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
admin1.namespaces().setBacklogQuota("pulsar/global/ns1", new BacklogQuota(1 * 1024 * 1024, policy));
Thread.sleep(200);
- DestinationName dest = DestinationName
+ TopicName dest = TopicName
.get(String.format("persistent://pulsar/global/ns1/%s-%d", policy, System.currentTimeMillis()));
// Producer on r1
@@ -796,7 +796,7 @@ public class ReplicatorTest extends ReplicatorTestBase {
@Test(timeOut = 5000)
public void testCloseReplicatorStartProducer() throws Exception {
- DestinationName dest = DestinationName.get("persistent://pulsar/global/ns1/closeCursor");
+ TopicName dest = TopicName.get("persistent://pulsar/global/ns1/closeCursor");
// Producer on r1
MessageProducer producer1 = new MessageProducer(url1, dest);
// Consumer on r1
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 4320428..d71f484 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
@@ -43,7 +43,7 @@ import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConfiguration;
import org.apache.pulsar.client.api.PulsarClient;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.apache.pulsar.common.util.FutureUtil;
@@ -95,7 +95,7 @@ public class ReplicatorTestBase {
return 60;
}
- public boolean isBrokerServicePurgeInactiveDestination() {
+ public boolean isBrokerServicePurgeInactiveTopic() {
return false;
}
@@ -122,7 +122,7 @@ public class ReplicatorTestBase {
config1.setWebServicePortTls(webServicePortTls1);
config1.setZookeeperServers("127.0.0.1:" + zkPort1);
config1.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config1.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config1.setBrokerServicePort(PortManager.nextFreePort());
@@ -156,7 +156,7 @@ public class ReplicatorTestBase {
config2.setWebServicePortTls(webServicePortTls2);
config2.setZookeeperServers("127.0.0.1:" + zkPort2);
config2.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config2.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config2.setBrokerServicePort(PortManager.nextFreePort());
@@ -190,7 +190,7 @@ public class ReplicatorTestBase {
config3.setWebServicePortTls(webServicePortTls3);
config3.setZookeeperServers("127.0.0.1:" + zkPort3);
config3.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config3.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config3.setBrokerServicePort(PortManager.nextFreePort());
@@ -265,7 +265,7 @@ public class ReplicatorTestBase {
PulsarClient client;
Producer producer;
- MessageProducer(URL url, final DestinationName dest) throws Exception {
+ MessageProducer(URL url, final TopicName dest) throws Exception {
this.url = url;
this.namespace = dest.getNamespace();
this.topicName = dest.toString();
@@ -276,7 +276,7 @@ public class ReplicatorTestBase {
}
- MessageProducer(URL url, final DestinationName dest, boolean batch) throws Exception {
+ MessageProducer(URL url, final TopicName dest, boolean batch) throws Exception {
this.url = url;
this.namespace = dest.getNamespace();
this.topicName = dest.toString();
@@ -337,11 +337,11 @@ public class ReplicatorTestBase {
final PulsarClient client;
final Consumer consumer;
- MessageConsumer(URL url, final DestinationName dest) throws Exception {
+ MessageConsumer(URL url, final TopicName dest) throws Exception {
this(url, dest, "sub-id");
}
- MessageConsumer(URL url, final DestinationName dest, String subId) throws Exception {
+ MessageConsumer(URL url, final TopicName dest, String subId) throws Exception {
this.url = url;
this.namespace = dest.getNamespace();
this.topicName = dest.toString();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
index 6c82e68..486f1cd 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java
@@ -89,7 +89,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.Policies;
@@ -176,7 +176,7 @@ public class ServerCnxTest {
namespaceService = mock(NamespaceService.class);
doReturn(namespaceService).when(pulsar).getNamespaceService();
doReturn(true).when(namespaceService).isServiceUnitOwned(any(NamespaceBundle.class));
- doReturn(true).when(namespaceService).isServiceUnitActive(any(DestinationName.class));
+ doReturn(true).when(namespaceService).isServiceUnitActive(any(TopicName.class));
setupMLAsyncCallbackMocks();
@@ -411,7 +411,7 @@ public class ServerCnxTest {
setChannelConnected();
// Force the case where the broker doesn't own any topic
- doReturn(false).when(namespaceService).isServiceUnitActive(any(DestinationName.class));
+ doReturn(false).when(namespaceService).isServiceUnitActive(any(TopicName.class));
// test PRODUCER failure case
ByteBuf clientCommand = Commands.newProducer(nonOwnedTopicName, 1 /* producer id */, 1 /* request id */,
@@ -502,7 +502,7 @@ public class ServerCnxTest {
doReturn(authorizationService).when(brokerService).getAuthorizationService();
doReturn(true).when(brokerService).isAuthorizationEnabled();
doReturn(false).when(authorizationProvider).isSuperUser(Mockito.anyString());
- doReturn(CompletableFuture.completedFuture(true)).when(authorizationProvider).checkPermission(any(DestinationName.class), Mockito.anyString(),
+ doReturn(CompletableFuture.completedFuture(true)).when(authorizationProvider).checkPermission(any(TopicName.class), Mockito.anyString(),
any(AuthAction.class));
resetChannel();
@@ -1238,8 +1238,8 @@ public class ServerCnxTest {
Policies policies = mock(Policies.class);
policies.encryption_required = true;
policies.clusterDispatchRate = Maps.newHashMap();
- doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
- doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
doReturn(zkDataCache).when(configCacheService).policiesCache();
// test success case: encrypted producer can connect
@@ -1266,8 +1266,8 @@ public class ServerCnxTest {
Policies policies = mock(Policies.class);
policies.encryption_required = true;
policies.clusterDispatchRate = Maps.newHashMap();
- doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
- doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
doReturn(zkDataCache).when(configCacheService).policiesCache();
// test failure case: unencrypted producer cannot connect
@@ -1296,8 +1296,8 @@ public class ServerCnxTest {
Policies policies = mock(Policies.class);
policies.encryption_required = true;
policies.clusterDispatchRate = Maps.newHashMap();
- doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
- doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
doReturn(zkDataCache).when(configCacheService).policiesCache();
ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */,
@@ -1331,8 +1331,8 @@ public class ServerCnxTest {
Policies policies = mock(Policies.class);
policies.encryption_required = true;
policies.clusterDispatchRate = Maps.newHashMap();
- doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
- doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, DestinationName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
+ doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace()));
doReturn(zkDataCache).when(configCacheService).policiesCache();
ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */,
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
index e950fbd..5288640 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZooKeeperClientAspectJTest.java
@@ -227,7 +227,7 @@ public class ZooKeeperClientAspectJTest {
BrokerService brokerService = pulsar.getBrokerService();
brokerService.updateRates();
- List<Metrics> metrics = brokerService.getDestinationMetrics();
+ List<Metrics> metrics = brokerService.getTopicMetrics();
AtomicDouble writeRate = new AtomicDouble();
AtomicDouble readRate = new AtomicDouble();
metrics.forEach(m -> {
@@ -250,7 +250,7 @@ public class ZooKeeperClientAspectJTest {
private Metrics getMetric(PulsarService pulsar, String dimension) {
BrokerService brokerService = pulsar.getBrokerService();
brokerService.updateRates();
- for (Metrics metric : brokerService.getDestinationMetrics()) {
+ for (Metrics metric : brokerService.getTopicMetrics()) {
if (dimension.equalsIgnoreCase(metric.getDimension("metric"))) {
return metric;
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
index 7292940..7c6f6a9 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java
@@ -307,17 +307,17 @@ public class AuthenticatedProducerConsumerTest extends ProducerConsumerBase {
String namespace = "my-property/use/my-ns";
admin.namespaces().createNamespace(namespace);
- String destination = "persistent://" + namespace + "1/topic1";
+ String topic = "persistent://" + namespace + "1/topic1";
// this will cause NPE and it should throw 500
mockZookKeeper.shutdown();
pulsar.getConfiguration().setSuperUserRoles(Sets.newHashSet());
try {
- admin.persistentTopics().getPartitionedTopicMetadata(destination);
+ admin.persistentTopics().getPartitionedTopicMetadata(topic);
} catch (PulsarAdminException e) {
Assert.assertTrue(e.getCause() instanceof InternalServerErrorException);
}
try {
- admin.lookups().lookupDestination(destination);
+ admin.lookups().lookupTopic(topic);
} catch (PulsarAdminException e) {
Assert.assertTrue(e.getCause() instanceof InternalServerErrorException);
}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
index 465cc6d..1eb27db 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
@@ -39,7 +39,7 @@ import org.apache.pulsar.broker.authorization.AuthorizationProvider;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.cache.ConfigurationCacheService;
import org.apache.pulsar.client.admin.PulsarAdmin;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -86,13 +86,13 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
/**
* It verifies plugable authorization service
- *
+ *
* <pre>
* 1. Client passes correct authorization plugin-name + correct auth role: SUCCESS
* 2. Client passes correct authorization plugin-name + incorrect auth-role: FAIL
* 3. Client passes incorrect authorization plugin-name + correct auth-role: FAIL
* </pre>
- *
+ *
* @throws Exception
*/
@Test
@@ -197,14 +197,14 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
setup();
AuthorizationService authorizationService = new AuthorizationService(conf, null);
- DestinationName destination = DestinationName.get("persistent://prop/cluster/ns/t1");
+ TopicName topicName = TopicName.get("persistent://prop/cluster/ns/t1");
String role = "test-role";
- Assert.assertFalse(authorizationService.canProduce(destination, role, null));
- Assert.assertFalse(authorizationService.canConsume(destination, role, null, "sub1"));
+ Assert.assertFalse(authorizationService.canProduce(topicName, role, null));
+ Assert.assertFalse(authorizationService.canConsume(topicName, role, null, "sub1"));
authorizationService
- .grantPermissionAsync(destination, null, role, "auth-json").get();
- Assert.assertTrue(authorizationService.canProduce(destination, role, null));
- Assert.assertTrue(authorizationService.canConsume(destination, role, null, "sub1"));
+ .grantPermissionAsync(topicName, null, role, "auth-json").get();
+ Assert.assertTrue(authorizationService.canProduce(topicName, role, null));
+ Assert.assertTrue(authorizationService.canConsume(topicName, role, null, "sub1"));
log.info("-- Exiting {} test --", methodName);
}
@@ -217,17 +217,17 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
setup();
AuthorizationService authorizationService = new AuthorizationService(conf, null);
- DestinationName destination = DestinationName.get("persistent://prop/cluster/ns/t1");
+ TopicName topicName = TopicName.get("persistent://prop/cluster/ns/t1");
String role = "test-role";
authorizationService
- .grantPermissionAsync(destination, null, role, "auth-json")
+ .grantPermissionAsync(topicName, null, role, "auth-json")
.get();
Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authDataJson, "auth-json");
Assert.assertTrue(
- authorizationService.canProduce(destination, role, new AuthenticationDataCommand("prod-auth")));
+ authorizationService.canProduce(topicName, role, new AuthenticationDataCommand("prod-auth")));
Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authenticationData.getCommandData(),
"prod-auth");
- Assert.assertTrue(authorizationService.canConsume(destination, role, new AuthenticationDataCommand("cons-auth"),
+ Assert.assertTrue(authorizationService.canConsume(topicName, role, new AuthenticationDataCommand("cons-auth"),
"sub1"));
Assert.assertEquals(TestAuthorizationProviderWithGrantPermission.authenticationData.getCommandData(),
"cons-auth");
@@ -324,19 +324,19 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
}
@Override
- public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
return CompletableFuture.completedFuture(clientRole.equals(role));
}
@Override
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
return CompletableFuture.completedFuture(clientRole.equals(role));
}
@Override
- public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
return CompletableFuture.completedFuture(clientRole.equals(role));
}
@@ -348,7 +348,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
}
@Override
- public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions,
+ public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions,
String role, String authenticationData) {
return CompletableFuture.completedFuture(null);
}
@@ -361,19 +361,19 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
public static class TestAuthorizationProvider2 extends TestAuthorizationProvider {
@Override
- public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
return CompletableFuture.completedFuture(true);
}
@Override
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
return CompletableFuture.completedFuture(false);
}
@Override
- public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
return CompletableFuture.completedFuture(true);
}
@@ -382,7 +382,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
public static class TestAuthorizationProviderWithSubscriptionPrefix extends TestAuthorizationProvider {
@Override
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
CompletableFuture<Boolean> future = new CompletableFuture<>();
if (isNotBlank(subscription)) {
@@ -404,21 +404,21 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
static String authDataJson;
@Override
- public CompletableFuture<Boolean> canProduceAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canProduceAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
this.authenticationData = authenticationData;
return CompletableFuture.completedFuture(grantRoles.contains(role));
}
@Override
- public CompletableFuture<Boolean> canConsumeAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData, String subscription) {
this.authenticationData = authenticationData;
return CompletableFuture.completedFuture(grantRoles.contains(role));
}
@Override
- public CompletableFuture<Boolean> canLookupAsync(DestinationName destination, String role,
+ public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
this.authenticationData = authenticationData;
return CompletableFuture.completedFuture(grantRoles.contains(role));
@@ -433,7 +433,7 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
}
@Override
- public CompletableFuture<Void> grantPermissionAsync(DestinationName topicname, Set<AuthAction> actions,
+ public CompletableFuture<Void> grantPermissionAsync(TopicName topicname, Set<AuthAction> actions,
String role, String authData) {
this.authDataJson = authData;
grantRoles.add(role);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
index 80d2d00..c72aa9e 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java
@@ -72,7 +72,7 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.auth.AuthenticationTls;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.ServiceUnitId;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
@@ -310,12 +310,12 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
log.info("-- Starting {} test --", methodName);
int numPartitions = 8;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setSubscriptionType(SubscriptionType.Exclusive);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
/**** start broker-2 ****/
ServiceConfiguration conf2 = new ServiceConfiguration();
@@ -348,9 +348,9 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
for (int i = 0; i < 20; i++) {
String message = "my-message-" + i;
@@ -371,7 +371,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
pulsar2.close();
loadManager2 = null;
@@ -846,8 +846,8 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
final String unsplitBundle = namespace + "/0x00000000_0xffffffff";
assertTrue(serviceUnits1.contains(unsplitBundle));
// broker-2 should have this bundle into the cache
- DestinationName destination = DestinationName.get(topic1);
- NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(destination);
+ TopicName topicName = TopicName.get(topic1);
+ NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(topicName);
assertEquals(bundleInBroker2.toString(), unsplitBundle);
// (5) Split the bundle for topic-1
@@ -856,7 +856,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
// (6) Broker-2 should get the watch and update bundle cache
final int retry = 5;
for (int i = 0; i < retry; i++) {
- if (pulsar2.getNamespaceService().getBundle(destination).equals(bundleInBroker2) && i != retry - 1) {
+ if (pulsar2.getNamespaceService().getBundle(topicName).equals(bundleInBroker2) && i != retry - 1) {
Thread.sleep(200);
} else {
break;
@@ -868,7 +868,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
Consumer consumer2 = pulsarClient.subscribe(topic2, "my-subscriber-name", new ConsumerConfiguration());
NamespaceBundle bundleInBroker1AfterSplit = pulsar2.getNamespaceService()
- .getBundle(DestinationName.get(topic2));
+ .getBundle(TopicName.get(topic2));
assertFalse(bundleInBroker1AfterSplit.equals(unsplitBundle));
consumer1.close();
@@ -955,8 +955,8 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
final String unsplitBundle = namespace + "/0x00000000_0xffffffff";
assertTrue(serviceUnits1.contains(unsplitBundle));
// broker-2 should have this bundle into the cache
- DestinationName destination = DestinationName.get(topic1);
- NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(destination);
+ TopicName topicName = TopicName.get(topic1);
+ NamespaceBundle bundleInBroker2 = pulsar2.getNamespaceService().getBundle(topicName);
assertEquals(bundleInBroker2.toString(), unsplitBundle);
// update broker-1 bundle report to zk
@@ -985,7 +985,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
// (6) Broker-2 should get the watch and update bundle cache
final int retry = 5;
for (int i = 0; i < retry; i++) {
- if (pulsar2.getNamespaceService().getBundle(destination).equals(bundleInBroker2) && i != retry - 1) {
+ if (pulsar2.getNamespaceService().getBundle(topicName).equals(bundleInBroker2) && i != retry - 1) {
Thread.sleep(200);
} else {
break;
@@ -997,7 +997,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
Consumer consumer2 = pulsarClient.subscribe(topic2, "my-subscriber-name", new ConsumerConfiguration());
NamespaceBundle bundleInBroker1AfterSplit = pulsar2.getNamespaceService()
- .getBundle(DestinationName.get(topic2));
+ .getBundle(TopicName.get(topic2));
assertFalse(bundleInBroker1AfterSplit.equals(unsplitBundle));
consumer1.close();
@@ -1018,7 +1018,7 @@ public class BrokerServiceLookupTest extends ProducerConsumerBase {
final String namespace = "my-ns";
final String topicName = "my-partitioned";
final int totalPartitions = 10;
- final DestinationName dest = DestinationName.get("persistent", property, cluster, namespace, topicName);
+ final TopicName dest = TopicName.get("persistent", property, cluster, namespace, topicName);
admin.clusters().createCluster(cluster,
new ClusterData("http://127.0.0.1:" + BROKER_WEBSERVICE_PORT, null, null, null));
admin.properties().createProperty(property,
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
index 5f45bc4..fefe8e7 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java
@@ -51,7 +51,7 @@ import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.ProducerImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
@@ -203,7 +203,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
// Ensure all partitions exist
for (int i = 0; i < numPartitions; i++) {
- DestinationName partition = DestinationName.get(topic).getPartition(i);
+ TopicName partition = TopicName.get(topic).getPartition(i);
assertNotNull(pulsar.getBrokerService().getTopicReference(partition.toString()));
}
@@ -494,7 +494,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
NonPersistentTopicStats stats;
SubscriptionStats subStats;
- DestinationName dest = DestinationName.get(globalTopicName);
+ TopicName dest = TopicName.get(globalTopicName);
PulsarClient client1 = PulsarClient.create(replication.url1.toString(), new ClientConfiguration());
PulsarClient client2 = PulsarClient.create(replication.url2.toString(), new ClientConfiguration());
@@ -652,7 +652,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
manager.start();
loadManagerRef.set(manager);
- NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+ NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
LoadManager loadManager = pulsar.getLoadManager().get();
ResourceUnit broker = null;
try {
@@ -741,7 +741,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
manager.start();
loadManagerRef.set(manager);
- NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(DestinationName.get(topicName));
+ NamespaceBundle fdqn = pulsar.getNamespaceService().getBundle(TopicName.get(topicName));
LoadManager loadManager = pulsar.getLoadManager().get();
ResourceUnit broker = null;
try {
@@ -865,7 +865,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
return 60;
}
- public boolean isBrokerServicePurgeInactiveDestination() {
+ public boolean isBrokerServicePurgeInactiveTopic() {
return false;
}
@@ -891,7 +891,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
config1.setWebServicePort(webServicePort1);
config1.setZookeeperServers("127.0.0.1:" + zkPort1);
config1.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config1.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config1.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config1.setBrokerServicePort(PortManager.nextFreePort());
@@ -917,7 +917,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
config2.setAdvertisedAddress("localhost");
config2.setZookeeperServers("127.0.0.1:" + zkPort2);
config2.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config2.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config2.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config2.setBrokerServicePort(PortManager.nextFreePort());
@@ -943,7 +943,7 @@ public class NonPersistentTopicTest extends ProducerConsumerBase {
config3.setAdvertisedAddress("localhost");
config3.setZookeeperServers("127.0.0.1:" + zkPort3);
config3.setGlobalZookeeperServers("127.0.0.1:" + globalZKPort + "/foo");
- config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveDestination());
+ config3.setBrokerDeleteInactiveTopicsEnabled(isBrokerServicePurgeInactiveTopic());
config3.setBrokerServicePurgeInactiveFrequencyInSeconds(
inSec(getBrokerServicePurgeInactiveFrequency(), TimeUnit.SECONDS));
config3.setBrokerServicePort(PortManager.nextFreePort());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
index dbe454d..3be9edc 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java
@@ -40,7 +40,7 @@ import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.PartitionedProducerImpl;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
@@ -78,18 +78,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
log.info("-- Starting {} test --", methodName);
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setSubscriptionType(SubscriptionType.Exclusive);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
@@ -110,7 +110,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
@@ -121,16 +121,15 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
int numPartitions = 4;
final String specialCharacter = "! * ' ( ) ; : @ & = + $ , /\\ ? % # [ ]";
- final String topicName = "my-partitionedtopic1" + specialCharacter;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/" + topicName);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1" + specialCharacter);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
// Try to create producer which does lookup and create connection with broker
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
producer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
@@ -139,18 +138,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
log.info("-- Starting {} test --", methodName);
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic2");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic2");
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setSubscriptionType(SubscriptionType.Exclusive);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.SinglePartition);
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
@@ -173,7 +172,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
@@ -183,17 +182,17 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
log.info("-- Starting {} test --", methodName);
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic3");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic3");
String dummyKey1 = "dummykey1";
String dummyKey2 = "dummykey2";
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setSubscriptionType(SubscriptionType.Exclusive);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
- Producer producer = pulsarClient.createProducer(dn.toString());
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Producer producer = pulsarClient.createProducer(topicName.toString());
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
Message msg = null;
for (int i = 0; i < 5; i++) {
@@ -221,7 +220,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
@@ -240,13 +239,13 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
log.info("-- Starting {} test --", methodName);
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic4");
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic4");
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ConsumerConfiguration consumerConf = new ConsumerConfiguration();
consumerConf.setSubscriptionType(SubscriptionType.Exclusive);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-subscriber-name", consumerConf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-subscriber-name", consumerConf);
try {
Message msg = MessageBuilder.create().setContent("InvalidMessage".getBytes()).build();
@@ -271,7 +270,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
// ok
}
- Producer producer = pulsarClient.createProducer(dn.toString());
+ Producer producer = pulsarClient.createProducer(topicName.toString());
producer.close();
try {
@@ -281,7 +280,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
// ok
}
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
}
@@ -289,8 +288,8 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
public void testSillyUser() throws Exception {
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic5");
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic5");
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
Producer producer = null;
@@ -311,7 +310,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
}
try {
- producer = pulsarClient.createProducer(dn.toString(), null);
+ producer = pulsarClient.createProducer(topicName.toString(), null);
Assert.fail("should fail");
} catch (PulsarClientException e) {
Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException);
@@ -320,15 +319,15 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
ConsumerConfiguration consumerConf = new ConsumerConfiguration();
try {
- consumer = pulsarClient.subscribe(dn.toString(), "my-subscriber-name", null);
+ consumer = pulsarClient.subscribe(topicName.toString(), "my-subscriber-name", null);
Assert.fail("Should fail");
} catch (PulsarClientException e) {
Assert.assertTrue(e instanceof PulsarClientException.InvalidConfigurationException);
}
try {
- producer = pulsarClient.createProducer(dn.toString());
- consumer = pulsarClient.subscribe(dn.toString(), "my-sub");
+ producer = pulsarClient.createProducer(topicName.toString());
+ consumer = pulsarClient.subscribe(topicName.toString(), "my-sub");
producer.send("message1".getBytes());
producer.send("message2".getBytes());
Message msg1 = consumer.receive();
@@ -343,25 +342,25 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
consumer.close();
}
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
}
@Test(timeOut = 30000)
public void testDeletePartitionedTopic() throws Exception {
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic6");
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic6");
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
- Producer producer = pulsarClient.createProducer(dn.toString());
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-sub");
+ Producer producer = pulsarClient.createProducer(topicName.toString());
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-sub");
consumer.unsubscribe();
consumer.close();
producer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
- Producer producer1 = pulsarClient.createProducer(dn.toString());
+ Producer producer1 = pulsarClient.createProducer(topicName.toString());
if (producer1 instanceof PartitionedProducerImpl) {
Assert.fail("should fail since partitioned topic was deleted");
}
@@ -376,18 +375,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
final Set<String> consumeMsgs = Sets.newHashSet();
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setSubscriptionType(SubscriptionType.Shared);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
// produce messages
for (int i = 0; i < totalMsg; i++) {
@@ -413,7 +412,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
@@ -427,18 +426,18 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
final Set<String> consumeMsgs = Sets.newHashSet();
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/my-partitionedtopic1");
ConsumerConfiguration conf = new ConsumerConfiguration();
conf.setReceiverQueueSize(1);
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
ProducerConfiguration producerConf = new ProducerConfiguration();
producerConf.setMessageRoutingMode(MessageRoutingMode.RoundRobinPartition);
- Producer producer = pulsarClient.createProducer(dn.toString(), producerConf);
+ Producer producer = pulsarClient.createProducer(topicName.toString(), producerConf);
- Consumer consumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber", conf);
+ Consumer consumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber", conf);
// produce messages
for (int i = 0; i < totalMsg; i++) {
@@ -464,7 +463,7 @@ public class PartitionedProducerConsumerTest extends ProducerConsumerBase {
producer.close();
consumer.unsubscribe();
consumer.close();
- admin.persistentTopics().deletePartitionedTopic(dn.toString());
+ admin.persistentTopics().deletePartitionedTopic(topicName.toString());
log.info("-- Exiting {} test --", methodName);
}
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 9635c43..33d1063 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
@@ -57,7 +57,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.api.PulsarDecoder;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -2152,9 +2152,9 @@ public class SimpleProducerConsumerTest extends ProducerConsumerBase {
// (2) Partitioned-consumer
int numPartitions = 4;
- DestinationName dn = DestinationName.get("persistent://my-property/use/my-ns/failAsyncReceive");
- admin.persistentTopics().createPartitionedTopic(dn.toString(), numPartitions);
- Consumer partitionedConsumer = pulsarClient.subscribe(dn.toString(), "my-partitioned-subscriber",
+ TopicName topicName = TopicName.get("persistent://my-property/use/my-ns/failAsyncReceive");
+ admin.persistentTopics().createPartitionedTopic(topicName.toString(), numPartitions);
+ Consumer partitionedConsumer = pulsarClient.subscribe(topicName.toString(), "my-partitioned-subscriber",
new ConsumerConfiguration());
partitionedConsumer.close();
// receive messages
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
index 5454bb7..8cf5618 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
@@ -64,7 +64,7 @@ import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.impl.HandlerBase.State;
import org.apache.pulsar.common.api.PulsarHandler;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.RetentionPolicies;
@@ -126,12 +126,12 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
admin.namespaces().createNamespace(ns1);
admin.namespaces().createNamespace(ns2);
- final String dn1 = "persistent://" + ns1 + "/my-topic";
- final String dn2 = "persistent://" + ns2 + "/my-topic";
- ConsumerImpl cons1 = (ConsumerImpl) pulsarClient.subscribe(dn1, "my-subscriber-name",
+ final String topic1 = "persistent://" + ns1 + "/my-topic";
+ final String topic2 = "persistent://" + ns2 + "/my-topic";
+ ConsumerImpl cons1 = (ConsumerImpl) pulsarClient.subscribe(topic1, "my-subscriber-name",
new ConsumerConfiguration());
- ProducerImpl prod1 = (ProducerImpl) pulsarClient.createProducer(dn1, new ProducerConfiguration());
- ProducerImpl prod2 = (ProducerImpl) pulsarClient.createProducer(dn2, new ProducerConfiguration());
+ ProducerImpl prod1 = (ProducerImpl) pulsarClient.createProducer(topic1, new ProducerConfiguration());
+ ProducerImpl prod2 = (ProducerImpl) pulsarClient.createProducer(topic2, new ProducerConfiguration());
ConsumerImpl consumer1 = spy(cons1);
doAnswer(invocationOnMock -> cons1.getState()).when(consumer1).getState();
doAnswer(invocationOnMock -> cons1.getClientCnx()).when(consumer1).getClientCnx();
@@ -178,8 +178,8 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
// disable this broker to avoid any new requests
pulsar.getLoadManager().get().disableBroker();
- NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn1));
- NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(DestinationName.get(dn2));
+ NamespaceBundle bundle1 = pulsar.getNamespaceService().getBundle(TopicName.get(topic1));
+ NamespaceBundle bundle2 = pulsar.getNamespaceService().getBundle(TopicName.get(topic2));
// unload ns-bundle:1
pulsar.getNamespaceService().unloadNamespaceBundle((NamespaceBundle) bundle1);
@@ -245,13 +245,13 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
admin.namespaces().createNamespace(ns1);
admin.namespaces().createNamespace(ns2);
- final String dn1 = "persistent://" + ns1 + "/my-topic";
- final String dn2 = "persistent://" + ns2 + "/my-topic";
+ final String topic1 = "persistent://" + ns1 + "/my-topic";
+ final String topic2 = "persistent://" + ns2 + "/my-topic";
- ConsumerImpl consumer1 = (ConsumerImpl) pulsarClient.subscribe(dn1, "my-subscriber-name",
+ ConsumerImpl consumer1 = (ConsumerImpl) pulsarClient.subscribe(topic1, "my-subscriber-name",
new ConsumerConfiguration());
- ProducerImpl producer1 = (ProducerImpl) pulsarClient.createProducer(dn1, new ProducerConfiguration());
- ProducerImpl producer2 = (ProducerImpl) pulsarClient.createProducer(dn2, new ProducerConfiguration());
+ ProducerImpl producer1 = (ProducerImpl) pulsarClient.createProducer(topic1, new ProducerConfiguration());
+ ProducerImpl producer2 = (ProducerImpl) pulsarClient.createProducer(topic2, new ProducerConfiguration());
// unload all other namespace
pulsar.getBrokerService().close();
@@ -376,7 +376,7 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
@Test(timeOut = 10000, dataProvider = "subType")
public void testResetCursor(SubscriptionType subType) throws Exception {
final RetentionPolicies policy = new RetentionPolicies(60, 52 * 1024);
- final DestinationName destName = DestinationName.get("persistent://my-property/use/my-ns/unacked-topic");
+ final TopicName destName = TopicName.get("persistent://my-property/use/my-ns/unacked-topic");
final int warmup = 20;
final int testSize = 150;
final List<Message> received = new ArrayList<Message>();
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
index e86f516..a577741 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundleTest.java
@@ -29,7 +29,7 @@ import static org.testng.Assert.fail;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -189,20 +189,20 @@ public class NamespaceBundleTest {
@Test
public void testIncludes() throws Exception {
- DestinationName dn = DestinationName.get("persistent://pulsar/use/ns1/topic-1");
- Long hashKey = factory.getLongHashCode(dn.toString());
+ TopicName topicName = TopicName.get("persistent://pulsar/use/ns1/topic-1");
+ Long hashKey = factory.getLongHashCode(topicName.toString());
Long upper = Math.max(hashKey + 1, NamespaceBundles.FULL_UPPER_BOUND);
BoundType upperType = upper.equals(NamespaceBundles.FULL_UPPER_BOUND) ? BoundType.CLOSED : BoundType.OPEN;
- NamespaceBundle bundle = factory.getBundle(dn.getNamespaceObject(),
+ NamespaceBundle bundle = factory.getBundle(topicName.getNamespaceObject(),
Range.range(hashKey / 2, BoundType.CLOSED, upper, upperType));
- assertTrue(bundle.includes(dn));
+ assertTrue(bundle.includes(topicName));
bundle = factory.getBundle(NamespaceName.get("pulsar/use/ns1"),
Range.range(upper, BoundType.CLOSED, NamespaceBundles.FULL_UPPER_BOUND, BoundType.CLOSED));
- assertTrue(!bundle.includes(dn));
+ assertTrue(!bundle.includes(topicName));
NamespaceBundle otherBundle = factory.getBundle(NamespaceName.get("pulsar/use/ns2"),
Range.range(0l, BoundType.CLOSED, 0x40000000L, BoundType.OPEN));
- assertTrue(!otherBundle.includes(dn));
+ assertTrue(!otherBundle.includes(topicName));
}
@Test
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
index 4c2bebe..310a834 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/NamespaceBundlesTest.java
@@ -37,7 +37,7 @@ import java.util.concurrent.CompletableFuture;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceBundleFactory;
import org.apache.pulsar.common.naming.NamespaceBundles;
@@ -143,19 +143,19 @@ public class NamespaceBundlesTest {
partitions.add(0xc0000000l);
partitions.add(0xffffffffl);
NamespaceBundles bundles = new NamespaceBundles(NamespaceName.get("pulsar/global/ns1"), partitions, factory);
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
- NamespaceBundle bundle = bundles.findBundle(dn);
- assertTrue(bundle.includes(dn));
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
+ NamespaceBundle bundle = bundles.findBundle(topicName);
+ assertTrue(bundle.includes(topicName));
- dn = DestinationName.get("persistent://pulsar/use/ns2/topic-2");
+ topicName = TopicName.get("persistent://pulsar/use/ns2/topic-2");
try {
- bundles.findBundle(dn);
+ bundles.findBundle(topicName);
fail("Should have failed due to mismatched namespace name");
} catch (IllegalArgumentException iae) {
// OK, expected
}
- Long hashKey = factory.getLongHashCode(dn.toString());
+ Long hashKey = factory.getLongHashCode(topicName.toString());
// The following code guarantees that we have at least two ranges after the hashKey till the end
SortedSet<Long> tailSet = partitions.tailSet(hashKey);
tailSet.add(hashKey);
@@ -165,8 +165,8 @@ public class NamespaceBundlesTest {
SortedSet<Long> newPar = tailSet.tailSet(iter.next());
try {
- bundles = new NamespaceBundles(dn.getNamespaceObject(), newPar, factory);
- bundles.findBundle(dn);
+ bundles = new NamespaceBundles(topicName.getNamespaceObject(), newPar, factory);
+ bundles.findBundle(topicName);
fail("Should have failed due to out-of-range");
} catch (ArrayIndexOutOfBoundsException iae) {
// OK, expected
@@ -176,9 +176,9 @@ public class NamespaceBundlesTest {
@Test
public void testsplitBundles() throws Exception {
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = factory.getBundles(nsname);
- NamespaceBundle bundle = bundles.findBundle(dn);
+ NamespaceBundle bundle = bundles.findBundle(topicName);
final int numberSplitBundles = 4;
// (1) split in 4
Pair<NamespaceBundles, List<NamespaceBundle>> splitBundles = factory.splitBundles(bundle, numberSplitBundles);
@@ -220,9 +220,9 @@ public class NamespaceBundlesTest {
public void testSplitBundleInTwo() throws Exception {
final int NO_BUNDLES = 2;
NamespaceName nsname = NamespaceName.get("pulsar/global/ns1");
- DestinationName dn = DestinationName.get("persistent://pulsar/global/ns1/topic-1");
+ TopicName topicName = TopicName.get("persistent://pulsar/global/ns1/topic-1");
NamespaceBundles bundles = factory.getBundles(nsname);
- NamespaceBundle bundle = bundles.findBundle(dn);
+ NamespaceBundle bundle = bundles.findBundle(topicName);
// (1) split : [0x00000000,0xffffffff] => [0x00000000_0x7fffffff,0x7fffffff_0xffffffff]
Pair<NamespaceBundles, List<NamespaceBundle>> splitBundles = factory.splitBundles(bundle, NO_BUNDLES);
assertNotNull(splitBundles);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
index 943c62d..4688e14 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java
@@ -29,7 +29,7 @@ import org.apache.bookkeeper.test.PortManager;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
@@ -80,7 +80,7 @@ public class ProxyAuthorizationTest extends MockedPulsarServiceBaseTest {
public void test() throws Exception {
AuthorizationService auth = service.getAuthorizationService();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
admin.clusters().updateCluster(configClusterName, new ClusterData());
admin.properties().createProperty("p1", new PropertyAdmin(Lists.newArrayList("role1"), Sets.newHashSet("c1")));
@@ -88,31 +88,31 @@ public class ProxyAuthorizationTest extends MockedPulsarServiceBaseTest {
admin.namespaces().createNamespace("p1/c1/ns1");
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.of(AuthAction.produce));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
admin.persistentTopics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role",
EnumSet.of(AuthAction.consume));
waitForChange();
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null), false);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null, null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds2"), "no-access-role", null, null), false);
- assertEquals(auth.canLookup(DestinationName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
+ assertEquals(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "no-access-role", null), false);
admin.namespaces().grantPermissionOnNamespace("p1/c1/ns1", "my-role", EnumSet.allOf(AuthAction.class));
waitForChange();
- assertEquals(auth.canProduce(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
- assertEquals(auth.canConsume(DestinationName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
+ assertEquals(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null), true);
+ assertEquals(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null), true);
admin.namespaces().deleteNamespace("p1/c1/ns1");
admin.properties().deleteProperty("p1");
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
index 6fe07c7..16ebd5e 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/BrokerStats.java
@@ -31,7 +31,7 @@ public interface BrokerStats {
/**
* Returns Monitoring metrics
- *
+ *
* @return
* @throws PulsarAdminException
*/
@@ -42,25 +42,25 @@ public interface BrokerStats {
* Requests JSON string server mbean dump
* <p>
* Notes: since we don't plan to introspect the response we avoid converting the response into POJO.
- *
+ *
* @return
* @throws PulsarAdminException
*/
JsonArray getMBeans() throws PulsarAdminException;
/**
- * Returns JSON string destination stats
+ * Returns JSON string topics stats
* <p>
* Notes: since we don't plan to introspect the response we avoid converting the response into POJO.
- *
+ *
* @return
* @throws PulsarAdminException
*/
- JsonObject getDestinations() throws PulsarAdminException;
+ JsonObject getTopics() throws PulsarAdminException;
JsonObject getPendingBookieOpsStats() throws PulsarAdminException;
AllocatorStats getAllocatorStats(String allocatorName) throws PulsarAdminException;
-
+
LoadManagerReport getLoadReport() throws PulsarAdminException;
}
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
index e578890..10d244b 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Lookup.java
@@ -19,25 +19,24 @@
package org.apache.pulsar.client.admin;
/**
- * This is an interface class to allow using command line tool to quickly lookup the broker serving the
- * destination.
+ * This is an interface class to allow using command line tool to quickly lookup the broker serving the topic.
*/
public interface Lookup {
/**
- * Lookup a destination
+ * Lookup a topic
*
- * @param destination
- * @return the broker URL that serves the destination
+ * @param topic
+ * @return the broker URL that serves the topic
*/
- public String lookupDestination(String destination) throws PulsarAdminException;
-
+ public String lookupTopic(String topic) throws PulsarAdminException;
+
/**
- * Get a bundle range of a destination
- *
- * @param destination
+ * Get a bundle range of a topic
+ *
+ * @param topic
* @return
* @throws PulsarAdminException
*/
- public String getBundleRange(String destination) throws PulsarAdminException;
+ public String getBundleRange(String topic) throws PulsarAdminException;
}
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
index 3c8d3d5..65ce200 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
@@ -90,9 +90,9 @@ public interface Namespaces {
List<String> getNamespaces(String property, String cluster) throws PulsarAdminException;
/**
- * Get the list of destinations.
+ * Get the list of topics.
* <p>
- * Get the list of all the destinations under a certain namespace.
+ * Get the list of all the topics under a certain namespace.
* <p>
* Response Example:
*
@@ -111,7 +111,7 @@ public interface Namespaces {
* @throws PulsarAdminException
* Unexpected error
*/
- List<String> getDestinations(String namespace) throws PulsarAdminException;
+ List<String> getTopics(String namespace) throws PulsarAdminException;
/**
* Get policies for a namespace.
@@ -420,8 +420,8 @@ public interface Namespaces {
* Unexpected error
*/
void setNamespaceMessageTTL(String namespace, int ttlInSeconds) throws PulsarAdminException;
-
-
+
+
/**
* Set anti-affinity group name for a namespace
* <p>
@@ -440,10 +440,10 @@ public interface Namespaces {
* Unexpected error
*/
void setNamespaceAntiAffinityGroup(String namespace, String namespaceAntiAffinityGroup) throws PulsarAdminException;
-
+
/**
* Get all namespaces that grouped with given anti-affinity group
- *
+ *
* @param property
* property is only used for authorization. Client has to be admin of any of the property to access this
* api api.
@@ -566,7 +566,7 @@ public interface Namespaces {
Map<BacklogQuota.BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(String namespace) throws PulsarAdminException;
/**
- * Set a backlog quota for all the destinations on a namespace.
+ * Set a backlog quota for all the topics on a namespace.
* <p>
* Set a backlog quota on a namespace.
* <p>
@@ -617,7 +617,7 @@ public interface Namespaces {
public void removeBacklogQuota(String namespace) throws PulsarAdminException;
/**
- * Set the persistence configuration for all the destinations on a namespace.
+ * Set the persistence configuration for all the topics on a namespace.
* <p>
* Set the persistence configuration on a namespace.
* <p>
@@ -695,7 +695,7 @@ public interface Namespaces {
public PersistencePolicies getPersistence(String namespace) throws PulsarAdminException;
/**
- * Set the retention configuration for all the destinations on a namespace.
+ * Set the retention configuration for all the topics on a namespace.
* <p/>
* Set the retention configuration on a namespace. This operation requires Pulsar super-user access.
* <p/>
@@ -825,7 +825,7 @@ public interface Namespaces {
DispatchRate getDispatchRate(String namespace) throws PulsarAdminException;
/**
- * Clear backlog for all destinations on a namespace
+ * Clear backlog for all topics on a namespace
*
* @param namespace
* @throws PulsarAdminException
@@ -834,7 +834,7 @@ public interface Namespaces {
void clearNamespaceBacklog(String namespace) throws PulsarAdminException;
/**
- * Clear backlog for a given subscription on all destinations on a namespace
+ * Clear backlog for a given subscription on all topics on a namespace
*
* @param namespace
* @param subscription
@@ -844,7 +844,7 @@ public interface Namespaces {
void clearNamespaceBacklogForSubscription(String namespace, String subscription) throws PulsarAdminException;
/**
- * Clear backlog for all destinations on a namespace bundle
+ * Clear backlog for all topics on a namespace bundle
*
* @param namespace
* @param bundle
@@ -854,7 +854,7 @@ public interface Namespaces {
void clearNamespaceBundleBacklog(String namespace, String bundle) throws PulsarAdminException;
/**
- * Clear backlog for a given subscription on all destinations on a namespace bundle
+ * Clear backlog for a given subscription on all topics on a namespace bundle
*
* @param namespace
* @param bundle
@@ -866,7 +866,7 @@ public interface Namespaces {
throws PulsarAdminException;
/**
- * Unsubscribes the given subscription on all destinations on a namespace
+ * Unsubscribes the given subscription on all topics on a namespace
*
* @param namespace
* @param subscription
@@ -875,7 +875,7 @@ public interface Namespaces {
void unsubscribeNamespace(String namespace, String subscription) throws PulsarAdminException;
/**
- * Unsubscribes the given subscription on all destinations on a namespace bundle
+ * Unsubscribes the given subscription on all topics on a namespace bundle
*
* @param namespace
* @param bundle
@@ -910,7 +910,7 @@ public interface Namespaces {
void setEncryptionRequiredStatus(String namespace, boolean encryptionRequired) throws PulsarAdminException;
/**
- * Set the given subscription auth mode on all destinations on a namespace
+ * Set the given subscription auth mode on all topics on a namespace
*
* @param namespace
* @param subscriptionAuthMode
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
index c21ea9a..2bef501 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/NonPersistentTopics.java
@@ -37,12 +37,12 @@ public interface NonPersistentTopics {
* Get metadata of a partitioned topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return Partitioned topic metadata
* @throws PulsarAdminException
*/
- PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException;
+ PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException;
/**
* Get metadata of a partitioned topic asynchronously.
@@ -50,11 +50,11 @@ public interface NonPersistentTopics {
* Get metadata of a partitioned topic asynchronously.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return a future that can be used to track when the partitioned topic metadata is returned
*/
- CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination);
+ CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic);
/**
* Get the stats for the topic.
@@ -117,8 +117,8 @@ public interface NonPersistentTopics {
*
* All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return the topic statistics
*
* @throws NotAuthorizedException
@@ -128,27 +128,27 @@ public interface NonPersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- NonPersistentTopicStats getStats(String destination) throws PulsarAdminException;
+ NonPersistentTopicStats getStats(String topic) throws PulsarAdminException;
/**
* Get the stats for the topic asynchronously. All the rates are computed over a 1 minute window and are relative
* the last completed 1 minute period.
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
*
* @return a future that can be used to track when the topic statistics are returned
*
*/
- CompletableFuture<NonPersistentTopicStats> getStatsAsync(String destination);
+ CompletableFuture<NonPersistentTopicStats> getStatsAsync(String topic);
/**
* Get the internal stats for the topic.
* <p>
* Access the internal state of the topic
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return the topic statistics
*
* @throws NotAuthorizedException
@@ -158,17 +158,17 @@ public interface NonPersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException;
+ PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException;
/**
* Get the internal stats for the topic asynchronously.
*
- * @param destination
- * Destination Name
+ * @param topic
+ * Topic Name
*
* @return a future that can be used to track when the internal topic statistics are returned
*/
- CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination);
+ CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic);
/**
* Create a partitioned topic.
@@ -176,13 +176,13 @@ public interface NonPersistentTopics {
* Create a partitioned topic. It needs to be called before creating a producer for a partitioned topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of partitions to create of the topic
* @throws PulsarAdminException
*/
- void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+ void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
/**
* Create a partitioned topic asynchronously.
@@ -191,44 +191,44 @@ public interface NonPersistentTopics {
* topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of partitions to create of the topic
* @return a future that can be used to track when the partitioned topic is created
*/
- CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions);
-
+ CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions);
+
/**
* Unload a topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
*
* @throws NotAuthorizedException
* Don't have admin permission
* @throws NotFoundException
- * Destination does not exist
+ * Topic does not exist
* @throws PulsarAdminException
* Unexpected error
*/
- void unload(String destination) throws PulsarAdminException;
+ void unload(String topic) throws PulsarAdminException;
/**
* Unload a topic asynchronously.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
*
* @return a future that can be used to track when the topic is unloaded
*/
- CompletableFuture<Void> unloadAsync(String destination);
+ CompletableFuture<Void> unloadAsync(String topic);
/**
* Get list of topics exist into given bundle
- *
+ *
* @param namespace
* @param bundleRange
* @return
@@ -239,7 +239,7 @@ public interface NonPersistentTopics {
/**
* Get list of topics exist into given bundle asynchronously.
- *
+ *
* @param namespace
* @param bundleRange
* @return
@@ -248,7 +248,7 @@ public interface NonPersistentTopics {
/**
* Get list of topics exist into given namespace
- *
+ *
* @param namespace
* @return
* @throws PulsarAdminException
@@ -257,7 +257,7 @@ public interface NonPersistentTopics {
/**
* Get list of topics exist into given namespace asynchronously.
- *
+ *
* @param namespace
* @param bundleRange
* @return
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
index 70642fe..b36caf8 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PersistentTopics.java
@@ -53,7 +53,7 @@ public interface PersistentTopics {
*
* @param namespace
* Namespace name
- * @return a list of destinations
+ * @return a list of topics
*
* @throws NotAuthorizedException
* Don't have admin permission
@@ -88,10 +88,10 @@ public interface PersistentTopics {
List<String> getPartitionedTopicList(String namespace) throws PulsarAdminException;
/**
- * Get permissions on a destination.
+ * Get permissions on a topic.
* <p>
- * Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the
- * namespace level combined (union) with any eventual specific permission set on the destination.
+ * Retrieve the effective permissions for a topic. These permissions are defined by the permissions set at the
+ * namespace level combined (union) with any eventual specific permission set on the topic.
* <p>
* Response Example:
*
@@ -102,9 +102,9 @@ public interface PersistentTopics {
* }</code>
* </pre>
*
- * @param destination
- * Destination url
- * @return a map of destinations an their permissions set
+ * @param topic
+ * Topic url
+ * @return a map of topics an their permissions set
*
* @throws NotAuthorizedException
* Don't have admin permission
@@ -113,12 +113,12 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- Map<String, Set<AuthAction>> getPermissions(String destination) throws PulsarAdminException;
+ Map<String, Set<AuthAction>> getPermissions(String topic) throws PulsarAdminException;
/**
- * Grant permission on a destination.
+ * Grant permission on a topic.
* <p>
- * Grant a new permission to a client role on a single destination.
+ * Grant a new permission to a client role on a single topic.
* <p>
* Request parameter example:
*
@@ -126,8 +126,8 @@ public interface PersistentTopics {
* <code>["produce", "consume"]</code>
* </pre>
*
- * @param destination
- * Destination url
+ * @param topic
+ * Topic url
* @param role
* Client role to which grant permission
* @param actions
@@ -142,16 +142,16 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void grantPermission(String destination, String role, Set<AuthAction> actions) throws PulsarAdminException;
+ void grantPermission(String topic, String role, Set<AuthAction> actions) throws PulsarAdminException;
/**
- * Revoke permissions on a destination.
+ * Revoke permissions on a topic.
* <p>
- * Revoke permissions to a client role on a single destination. If the permission was not set at the destination
- * level, but rather at the namespace level, this operation will return an error (HTTP status code 412).
+ * Revoke permissions to a client role on a single topic. If the permission was not set at the topic level, but
+ * rather at the namespace level, this operation will return an error (HTTP status code 412).
*
- * @param destination
- * Destination url
+ * @param topic
+ * Topic url
* @param role
* Client role to which remove permission
* @throws UniformInterfaceException
@@ -162,11 +162,11 @@ public interface PersistentTopics {
* @throws NotFound
* Namespace does not exist
* @throws PreconditionFailedException
- * Permissions are not set at the destination level
+ * Permissions are not set at the topic level
* @throws PulsarAdminException
* Unexpected error
*/
- void revokePermissions(String destination, String role) throws PulsarAdminException;
+ void revokePermissions(String topic, String role) throws PulsarAdminException;
/**
* Create a partitioned topic.
@@ -174,13 +174,13 @@ public interface PersistentTopics {
* Create a partitioned topic. It needs to be called before creating a producer for a partitioned topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of partitions to create of the topic
* @throws PulsarAdminException
*/
- void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+ void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
/**
* Create a partitioned topic asynchronously.
@@ -189,13 +189,13 @@ public interface PersistentTopics {
* topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of partitions to create of the topic
* @return a future that can be used to track when the partitioned topic is created
*/
- CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions);
+ CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions);
/**
* Update number of partitions of a non-global partitioned topic.
@@ -204,14 +204,14 @@ public interface PersistentTopics {
* number of partitions. Decrementing number of partitions requires deletion of topic which is not supported.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of new partitions of already exist partitioned-topic
*
* @return a future that can be used to track when the partitioned topic is updated
*/
- void updatePartitionedTopic(String destination, int numPartitions) throws PulsarAdminException;
+ void updatePartitionedTopic(String topic, int numPartitions) throws PulsarAdminException;
/**
* Update number of partitions of a non-global partitioned topic asynchronously.
@@ -220,14 +220,14 @@ public interface PersistentTopics {
* number of partitions. Decrementing number of partitions requires deletion of topic which is not supported.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @param numPartitions
* Number of new partitions of already exist partitioned-topic
*
* @return a future that can be used to track when the partitioned topic is updated
*/
- CompletableFuture<Void> updatePartitionedTopicAsync(String destination, int numPartitions);
+ CompletableFuture<Void> updatePartitionedTopicAsync(String topic, int numPartitions);
/**
* Get metadata of a partitioned topic.
@@ -235,12 +235,12 @@ public interface PersistentTopics {
* Get metadata of a partitioned topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return Partitioned topic metadata
* @throws PulsarAdminException
*/
- PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException;
+ PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException;
/**
* Get metadata of a partitioned topic asynchronously.
@@ -248,11 +248,11 @@ public interface PersistentTopics {
* Get metadata of a partitioned topic asynchronously.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return a future that can be used to track when the partitioned topic metadata is returned
*/
- CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination);
+ CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic);
/**
* Delete a partitioned topic.
@@ -260,11 +260,11 @@ public interface PersistentTopics {
* It will also delete all the partitions of the topic if it exists.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @throws PulsarAdminException
*/
- void deletePartitionedTopic(String destination) throws PulsarAdminException;
+ void deletePartitionedTopic(String topic) throws PulsarAdminException;
/**
* Delete a partitioned topic asynchronously.
@@ -272,11 +272,11 @@ public interface PersistentTopics {
* It will also delete all the partitions of the topic if it exists.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
* @return a future that can be used to track when the partitioned topic is deleted
*/
- CompletableFuture<Void> deletePartitionedTopicAsync(String destination);
+ CompletableFuture<Void> deletePartitionedTopicAsync(String topic);
/**
* Delete a topic.
@@ -284,19 +284,19 @@ public interface PersistentTopics {
* Delete a topic. The topic cannot be deleted if there's any active subscription or producer connected to the it.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * Topic name
*
* @throws NotAuthorizedException
* Don't have admin permission
* @throws NotFoundException
- * Destination does not exist
+ * Topic does not exist
* @throws PreconditionFailedException
* Topic has active subscriptions or producers
* @throws PulsarAdminException
* Unexpected error
*/
- void delete(String destination) throws PulsarAdminException;
+ void delete(String topic) throws PulsarAdminException;
/**
* Delete a topic asynchronously.
@@ -305,50 +305,50 @@ public interface PersistentTopics {
* connected to the it.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
*
* @return a future that can be used to track when the topic is deleted
*/
- CompletableFuture<Void> deleteAsync(String destination);
+ CompletableFuture<Void> deleteAsync(String topic);
/**
* Unload a topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
*
* @throws NotAuthorizedException
* Don't have admin permission
* @throws NotFoundException
- * Destination does not exist
+ * topic does not exist
* @throws PulsarAdminException
* Unexpected error
*/
- void unload(String destination) throws PulsarAdminException;
+ void unload(String topic) throws PulsarAdminException;
/**
* Unload a topic asynchronously.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
*
* @return a future that can be used to track when the topic is unloaded
*/
- CompletableFuture<Void> unloadAsync(String destination);
+ CompletableFuture<Void> unloadAsync(String topic);
/**
* Terminate the topic and prevent any more messages being published on it.
* <p>
* This
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return the message id of the last message that was published in the topic
*/
- CompletableFuture<MessageId> terminateTopicAsync(String destination);
+ CompletableFuture<MessageId> terminateTopicAsync(String topic);
/**
* Get the list of subscriptions.
@@ -356,8 +356,8 @@ public interface PersistentTopics {
* Get the list of persistent subscriptions for a given topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return the list of subscriptions
*
* @throws NotAuthorizedException
@@ -367,7 +367,7 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- List<String> getSubscriptions(String destination) throws PulsarAdminException;
+ List<String> getSubscriptions(String topic) throws PulsarAdminException;
/**
* Get the list of subscriptions asynchronously.
@@ -375,11 +375,11 @@ public interface PersistentTopics {
* Get the list of persistent subscriptions for a given topic.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return a future that can be used to track when the list of subscriptions is returned
*/
- CompletableFuture<List<String>> getSubscriptionsAsync(String destination);
+ CompletableFuture<List<String>> getSubscriptionsAsync(String topic);
/**
* Get the stats for the topic.
@@ -444,8 +444,8 @@ public interface PersistentTopics {
*
* All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return the topic statistics
*
* @throws NotAuthorizedException
@@ -455,27 +455,27 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- PersistentTopicStats getStats(String destination) throws PulsarAdminException;
+ PersistentTopicStats getStats(String topic) throws PulsarAdminException;
/**
* Get the stats for the topic asynchronously. All the rates are computed over a 1 minute window and are relative
* the last completed 1 minute period.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
*
* @return a future that can be used to track when the topic statistics are returned
*
*/
- CompletableFuture<PersistentTopicStats> getStatsAsync(String destination);
+ CompletableFuture<PersistentTopicStats> getStatsAsync(String topic);
/**
* Get the internal stats for the topic.
* <p>
* Access the internal state of the topic
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return the topic statistics
*
* @throws NotAuthorizedException
@@ -485,23 +485,23 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException;
+ PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException;
/**
* Get the internal stats for the topic asynchronously.
*
- * @param destination
- * Destination Name
+ * @param topic
+ * topic Name
*
* @return a future that can be used to track when the internal topic statistics are returned
*/
- CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination);
+ CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic);
/**
* Get a JSON representation of the topic metadata stored in ZooKeeper
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return the topic internal metadata
* @throws NotAuthorizedException
* Don't have admin permission
@@ -510,13 +510,13 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- JsonObject getInternalInfo(String destination) throws PulsarAdminException;
+ JsonObject getInternalInfo(String topic) throws PulsarAdminException;
/**
* Get a JSON representation of the topic metadata stored in ZooKeeper
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @return a future to receive the topic internal metadata
* @throws NotAuthorizedException
* Don't have admin permission
@@ -525,7 +525,7 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- CompletableFuture<JsonObject> getInternalInfoAsync(String destination);
+ CompletableFuture<JsonObject> getInternalInfoAsync(String topic);
/**
* Get the stats for the partitioned topic
@@ -584,8 +584,8 @@ public interface PersistentTopics {
*
* All the rates are computed over a 1 minute window and are relative the last completed 1 minute period.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param perPartition
*
* @return the partitioned topic statistics
@@ -597,18 +597,18 @@ public interface PersistentTopics {
* Unexpected error
*
*/
- PartitionedTopicStats getPartitionedStats(String destination, boolean perPartition) throws PulsarAdminException;
+ PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition) throws PulsarAdminException;
/**
* Get the stats for the partitioned topic asynchronously
*
- * @param destination
- * Destination Name
+ * @param topic
+ * topic Name
* @param perPartition
* flag to get stats per partition
* @return a future that can be used to track when the partitioned topic statistics are returned
*/
- CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String destination, boolean perPartition);
+ CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String topic, boolean perPartition);
/**
* Delete a subscription.
@@ -616,8 +616,8 @@ public interface PersistentTopics {
* Delete a persistent subscription from a topic. There should not be any active consumers on the subscription.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
*
@@ -630,7 +630,7 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void deleteSubscription(String destination, String subName) throws PulsarAdminException;
+ void deleteSubscription(String topic, String subName) throws PulsarAdminException;
/**
* Delete a subscription asynchronously.
@@ -638,22 +638,22 @@ public interface PersistentTopics {
* Delete a persistent subscription from a topic. There should not be any active consumers on the subscription.
* <p>
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
*
* @return a future that can be used to track when the subscription is deleted
*/
- CompletableFuture<Void> deleteSubscriptionAsync(String destination, String subName);
+ CompletableFuture<Void> deleteSubscriptionAsync(String topic, String subName);
/**
* Skip all messages on a topic subscription.
* <p>
* Completely clears the backlog on the subscription.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
*
@@ -664,27 +664,27 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void skipAllMessages(String destination, String subName) throws PulsarAdminException;
+ void skipAllMessages(String topic, String subName) throws PulsarAdminException;
/**
* Skip all messages on a topic subscription asynchronously.
* <p>
* Completely clears the backlog on the subscription.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
*
* @return a future that can be used to track when all the messages are skipped
*/
- CompletableFuture<Void> skipAllMessagesAsync(String destination, String subName);
+ CompletableFuture<Void> skipAllMessagesAsync(String topic, String subName);
/**
* Skip messages on a topic subscription.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param numMessages
@@ -697,13 +697,13 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void skipMessages(String destination, String subName, long numMessages) throws PulsarAdminException;
+ void skipMessages(String topic, String subName, long numMessages) throws PulsarAdminException;
/**
* Skip messages on a topic subscription asynchronously.
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param numMessages
@@ -711,13 +711,13 @@ public interface PersistentTopics {
*
* @return a future that can be used to track when the number of messages are skipped
*/
- CompletableFuture<Void> skipMessagesAsync(String destination, String subName, long numMessages);
+ CompletableFuture<Void> skipMessagesAsync(String topic, String subName, long numMessages);
/**
* Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param expireTimeInSeconds
@@ -725,51 +725,53 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- public void expireMessages(String destination, String subscriptionName, long expireTimeInSeconds) throws PulsarAdminException;
+ public void expireMessages(String topic, String subscriptionName, long expireTimeInSeconds)
+ throws PulsarAdminException;
/**
* Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription asynchronously
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param expireTimeInSeconds
* Expire messages older than time in seconds
* @return
*/
- public CompletableFuture<Void> expireMessagesAsync(String destination, String subscriptionName, long expireTimeInSeconds);
+ public CompletableFuture<Void> expireMessagesAsync(String topic, String subscriptionName,
+ long expireTimeInSeconds);
/**
* Expire all messages older than given N (expireTimeInSeconds) seconds for all subscriptions of the
* persistent-topic
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param expireTimeInSeconds
* Expire messages older than time in seconds
* @throws PulsarAdminException
* Unexpected error
*/
- public void expireMessagesForAllSubscriptions(String destination, long expireTimeInSeconds) throws PulsarAdminException;
-
+ public void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds)
+ throws PulsarAdminException;
/**
* Expire all messages older than given N (expireTimeInSeconds) seconds for all subscriptions of the
* persistent-topic asynchronously
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param expireTimeInSeconds
* Expire messages older than time in seconds
*/
- public CompletableFuture<Void> expireMessagesForAllSubscriptionsAsync(String destination, long expireTimeInSeconds);
+ public CompletableFuture<Void> expireMessagesForAllSubscriptionsAsync(String topic, long expireTimeInSeconds);
/**
* Peek messages from a topic subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param numMessages
@@ -782,26 +784,26 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- List<Message> peekMessages(String destination, String subName, int numMessages) throws PulsarAdminException;
+ List<Message> peekMessages(String topic, String subName, int numMessages) throws PulsarAdminException;
/**
* Peek messages from a topic subscription asynchronously
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param numMessages
* Number of messages
* @return a future that can be used to track when the messages are returned
*/
- CompletableFuture<List<Message>> peekMessagesAsync(String destination, String subName, int numMessages);
+ CompletableFuture<List<Message>> peekMessagesAsync(String topic, String subName, int numMessages);
/**
* Create a new subscription on a topic
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subscriptionName
* Subscription name
* @param messageId
@@ -817,27 +819,27 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void createSubscription(String destination, String subscriptionName, MessageId messageId)
+ void createSubscription(String topic, String subscriptionName, MessageId messageId)
throws PulsarAdminException;
/**
* Create a new subscription on a topic
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subscriptionName
* Subscription name
* @param messageId
* The {@link MessageId} on where to initialize the subscription. It could be {@link MessageId#latest},
* {@link MessageId#earliest} or a specific message id.
*/
- CompletableFuture<Void> createSubscriptionAsync(String destination, String subscriptionName, MessageId messageId);
+ CompletableFuture<Void> createSubscriptionAsync(String topic, String subscriptionName, MessageId messageId);
/**
* Reset cursor position on a topic subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param timestamp
@@ -852,25 +854,25 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void resetCursor(String destination, String subName, long timestamp) throws PulsarAdminException;
+ void resetCursor(String topic, String subName, long timestamp) throws PulsarAdminException;
/**
* Reset cursor position on a topic subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param timestamp
* reset subscription to position closest to time in ms since epoch
*/
- CompletableFuture<Void> resetCursorAsync(String destination, String subName, long timestamp);
+ CompletableFuture<Void> resetCursorAsync(String topic, String subName, long timestamp);
/**
* Reset cursor position on a topic subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param messageId
@@ -885,17 +887,17 @@ public interface PersistentTopics {
* @throws PulsarAdminException
* Unexpected error
*/
- void resetCursor(String destination, String subName, MessageId messageId) throws PulsarAdminException;
+ void resetCursor(String topic, String subName, MessageId messageId) throws PulsarAdminException;
/**
* Reset cursor position on a topic subscription
*
- * @param destination
- * Destination name
+ * @param topic
+ * topic name
* @param subName
* Subscription name
* @param MessageId
* reset subscription to messageId (or previous nearest messageId if given messageId is not valid)
*/
- CompletableFuture<Void> resetCursorAsync(String destination, String subName, MessageId messageId);
+ CompletableFuture<Void> resetCursorAsync(String topic, String subName, MessageId messageId);
}
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
index e801925..9be2516 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Properties.java
@@ -110,7 +110,7 @@ public interface Properties {
/**
* Delete an existing property.
* <p>
- * Delete a property and all namespaces and destinations under it.
+ * Delete a property and all namespaces and topics under it.
*
* @param property
* Property name
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
index 21f809d..dfa6579 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdmin.java
@@ -264,14 +264,14 @@ public class PulsarAdmin implements Closeable {
public ResourceQuotas resourceQuotas() {
return resourceQuotas;
}
-
+
/**
- * @return does a looks up for the broker serving the destination
+ * @return does a looks up for the broker serving the topic
*/
public Lookup lookups() {
return lookups;
}
-
+
/**
* @return the broker statics
*/
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
index 917fd65..82e1722 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java
@@ -75,7 +75,7 @@ public class BrokerStatsImpl extends BaseResource implements BrokerStats {
}
@Override
- public JsonObject getDestinations() throws PulsarAdminException {
+ public JsonObject getTopics() throws PulsarAdminException {
try {
String json = request(brokerStats.path("/destinations")).get(String.class);
return new Gson().fromJson(json, JsonObject.class);
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
index 4a943c9..677dd46 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/LookupImpl.java
@@ -25,7 +25,7 @@ import org.apache.pulsar.client.admin.Lookup;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Authentication;
import org.apache.pulsar.common.lookup.data.LookupData;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
public class LookupImpl extends BaseResource implements Lookup {
@@ -47,26 +47,26 @@ public class LookupImpl extends BaseResource implements Lookup {
}
@Override
- public String lookupDestination(String destination) throws PulsarAdminException {
+ public String lookupTopic(String topic) throws PulsarAdminException {
try {
- DestinationName destName = DestinationName.get(destination);
- return doDestinationLookup(v2lookup.path("/destination"), destName);
+ TopicName topicName = TopicName.get(topic);
+ return doTopicLookup(v2lookup.path("/destination"), topicName);
} catch (Exception e) {
throw getLookupApiException(e);
}
}
@Override
- public String getBundleRange(String destination) throws PulsarAdminException {
+ public String getBundleRange(String topic) throws PulsarAdminException {
try {
- DestinationName destName = DestinationName.get(destination);
+ TopicName destName = TopicName.get(topic);
return request(v2lookup.path("/destination").path(destName.getLookupName()).path("bundle")).get(String.class);
} catch (Exception e) {
throw getLookupApiException(e);
}
}
- private String doDestinationLookup(WebTarget lookupResource, DestinationName destName) throws PulsarAdminException {
+ private String doTopicLookup(WebTarget lookupResource, TopicName destName) throws PulsarAdminException {
LookupData lookupData = request(lookupResource.path(destName.getLookupName())).get(LookupData.class);
if (useTls) {
return lookupData.getBrokerUrlTls();
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
index 3f0c7e7..6db9ab0 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
@@ -72,7 +72,7 @@ public class NamespacesImpl extends BaseResource implements Namespaces {
}
@Override
- public List<String> getDestinations(String namespace) throws PulsarAdminException {
+ public List<String> getTopics(String namespace) throws PulsarAdminException {
try {
NamespaceName ns = NamespaceName.get(namespace);
return request(namespaces.path(ns.getProperty()).path(ns.getCluster()).path(ns.getLocalName())
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
index bfaa81f..48bfd09 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java
@@ -33,7 +33,7 @@ import javax.ws.rs.core.MediaType;
import org.apache.pulsar.client.admin.NonPersistentTopics;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Authentication;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.NonPersistentTopicStats;
@@ -49,9 +49,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+ public void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
try {
- createPartitionedTopicAsync(destination, numPartitions).get();
+ createPartitionedTopicAsync(topic, numPartitions).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -61,18 +61,18 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions) {
+ public CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions) {
checkArgument(numPartitions > 1, "Number of partitions should be more than 1");
- DestinationName ds = validateTopic(destination);
+ TopicName ds = validateTopic(topic);
return asyncPutRequest(
nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
}
@Override
- public PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException {
+ public PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException {
try {
- return getPartitionedTopicMetadataAsync(destination).get();
+ return getPartitionedTopicMetadataAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -82,8 +82,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
new InvocationCallback<PartitionedTopicMetadata>() {
@@ -102,9 +102,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public NonPersistentTopicStats getStats(String destination) throws PulsarAdminException {
+ public NonPersistentTopicStats getStats(String topic) throws PulsarAdminException {
try {
- return getStatsAsync(destination).get();
+ return getStatsAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -114,8 +114,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public CompletableFuture<NonPersistentTopicStats> getStatsAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<NonPersistentTopicStats> getStatsAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<NonPersistentTopicStats> future = new CompletableFuture<>();
asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("stats"),
new InvocationCallback<NonPersistentTopicStats>() {
@@ -134,9 +134,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException {
+ public PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException {
try {
- return getInternalStatsAsync(destination).get();
+ return getInternalStatsAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -146,8 +146,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PersistentTopicInternalStats> future = new CompletableFuture<>();
asyncGetRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internalStats"),
new InvocationCallback<PersistentTopicInternalStats>() {
@@ -166,9 +166,9 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public void unload(String destination) throws PulsarAdminException {
+ public void unload(String topic) throws PulsarAdminException {
try {
- unloadAsync(destination).get();
+ unloadAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -178,8 +178,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
@Override
- public CompletableFuture<Void> unloadAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> unloadAsync(String topic) {
+ TopicName ds = validateTopic(topic);
return asyncPutRequest(nonPersistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("unload"),
Entity.entity("", MediaType.APPLICATION_JSON));
}
@@ -246,11 +246,11 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste
}
/*
- * returns destination name with encoded Local Name
+ * returns topic name with encoded Local Name
*/
- private DestinationName validateTopic(String destination) {
+ private TopicName validateTopic(String topic) {
// Parsing will throw exception if name is not valid
- return DestinationName.get(destination);
+ return TopicName.get(topic);
}
}
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
index 01ede00..92bd242 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PersistentTopicsImpl.java
@@ -53,7 +53,7 @@ import org.apache.pulsar.common.api.Commands;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue;
import org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata;
-import org.apache.pulsar.common.naming.DestinationName;
+import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
import org.apache.pulsar.common.policies.data.AuthAction;
@@ -107,9 +107,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public Map<String, Set<AuthAction>> getPermissions(String destination) throws PulsarAdminException {
+ public Map<String, Set<AuthAction>> getPermissions(String topic) throws PulsarAdminException {
try {
- DestinationName ds = DestinationName.get(destination);
+ TopicName ds = TopicName.get(topic);
return request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions")).get(
new GenericType<Map<String, Set<AuthAction>>>() {
});
@@ -119,9 +119,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void grantPermission(String destination, String role, Set<AuthAction> actions) throws PulsarAdminException {
+ public void grantPermission(String topic, String role, Set<AuthAction> actions) throws PulsarAdminException {
try {
- DestinationName ds = DestinationName.get(destination);
+ TopicName ds = TopicName.get(topic);
request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions").path(role))
.post(Entity.entity(actions, MediaType.APPLICATION_JSON), ErrorData.class);
} catch (Exception e) {
@@ -130,9 +130,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void revokePermissions(String destination, String role) throws PulsarAdminException {
+ public void revokePermissions(String topic, String role) throws PulsarAdminException {
try {
- DestinationName ds = DestinationName.get(destination);
+ TopicName ds = TopicName.get(topic);
request(persistentTopics.path(ds.getNamespace()).path(ds.getLocalName()).path("permissions").path(role))
.delete(ErrorData.class);
} catch (Exception e) {
@@ -141,9 +141,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void createPartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+ public void createPartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
try {
- createPartitionedTopicAsync(destination, numPartitions).get();
+ createPartitionedTopicAsync(topic, numPartitions).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -153,18 +153,18 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> createPartitionedTopicAsync(String destination, int numPartitions) {
+ public CompletableFuture<Void> createPartitionedTopicAsync(String topic, int numPartitions) {
checkArgument(numPartitions > 1, "Number of partitions should be more than 1");
- DestinationName ds = validateTopic(destination);
+ TopicName ds = validateTopic(topic);
return asyncPutRequest(
persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
}
@Override
- public void updatePartitionedTopic(String destination, int numPartitions) throws PulsarAdminException {
+ public void updatePartitionedTopic(String topic, int numPartitions) throws PulsarAdminException {
try {
- updatePartitionedTopicAsync(destination, numPartitions).get();
+ updatePartitionedTopicAsync(topic, numPartitions).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -174,18 +174,18 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> updatePartitionedTopicAsync(String destination, int numPartitions) {
+ public CompletableFuture<Void> updatePartitionedTopicAsync(String topic, int numPartitions) {
checkArgument(numPartitions > 1, "Number of partitions must be more than 1");
- DestinationName ds = validateTopic(destination);
+ TopicName ds = validateTopic(topic);
return asyncPostRequest(
persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
Entity.entity(numPartitions, MediaType.APPLICATION_JSON));
}
@Override
- public PartitionedTopicMetadata getPartitionedTopicMetadata(String destination) throws PulsarAdminException {
+ public PartitionedTopicMetadata getPartitionedTopicMetadata(String topic) throws PulsarAdminException {
try {
- return getPartitionedTopicMetadataAsync(destination).get();
+ return getPartitionedTopicMetadataAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -195,8 +195,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadataAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitions"),
new InvocationCallback<PartitionedTopicMetadata>() {
@@ -215,9 +215,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void deletePartitionedTopic(String destination) throws PulsarAdminException {
+ public void deletePartitionedTopic(String topic) throws PulsarAdminException {
try {
- deletePartitionedTopicAsync(destination).get();
+ deletePartitionedTopicAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -227,16 +227,16 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> deletePartitionedTopicAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> deletePartitionedTopicAsync(String topic) {
+ TopicName ds = validateTopic(topic);
return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName())
.path("partitions"));
}
@Override
- public void delete(String destination) throws PulsarAdminException {
+ public void delete(String topic) throws PulsarAdminException {
try {
- deleteAsync(destination).get();
+ deleteAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -246,15 +246,15 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> deleteAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> deleteAsync(String topic) {
+ TopicName ds = validateTopic(topic);
return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()));
}
@Override
- public void unload(String destination) throws PulsarAdminException {
+ public void unload(String topic) throws PulsarAdminException {
try {
- unloadAsync(destination).get();
+ unloadAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -264,16 +264,16 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> unloadAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> unloadAsync(String topic) {
+ TopicName ds = validateTopic(topic);
return asyncPutRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("unload"),
Entity.entity("", MediaType.APPLICATION_JSON));
}
@Override
- public List<String> getSubscriptions(String destination) throws PulsarAdminException {
+ public List<String> getSubscriptions(String topic) throws PulsarAdminException {
try {
- return getSubscriptionsAsync(destination).get();
+ return getSubscriptionsAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -283,8 +283,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<List<String>> getSubscriptionsAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<List<String>> getSubscriptionsAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<List<String>> future = new CompletableFuture<>();
asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscriptions"),
new InvocationCallback<List<String>>() {
@@ -303,9 +303,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public PersistentTopicStats getStats(String destination) throws PulsarAdminException {
+ public PersistentTopicStats getStats(String topic) throws PulsarAdminException {
try {
- return getStatsAsync(destination).get();
+ return getStatsAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -315,8 +315,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<PersistentTopicStats> getStatsAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<PersistentTopicStats> getStatsAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PersistentTopicStats> future = new CompletableFuture<>();
asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("stats"),
new InvocationCallback<PersistentTopicStats>() {
@@ -335,9 +335,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public PersistentTopicInternalStats getInternalStats(String destination) throws PulsarAdminException {
+ public PersistentTopicInternalStats getInternalStats(String topic) throws PulsarAdminException {
try {
- return getInternalStatsAsync(destination).get();
+ return getInternalStatsAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -347,8 +347,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<PersistentTopicInternalStats> getInternalStatsAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PersistentTopicInternalStats> future = new CompletableFuture<>();
asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internalStats"),
new InvocationCallback<PersistentTopicInternalStats>() {
@@ -367,9 +367,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public JsonObject getInternalInfo(String destination) throws PulsarAdminException {
+ public JsonObject getInternalInfo(String topic) throws PulsarAdminException {
try {
- return getInternalInfoAsync(destination).get();
+ return getInternalInfoAsync(topic).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -379,8 +379,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<JsonObject> getInternalInfoAsync(String destination) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<JsonObject> getInternalInfoAsync(String topic) {
+ TopicName ds = validateTopic(topic);
final CompletableFuture<JsonObject> future = new CompletableFuture<>();
asyncGetRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("internal-info"),
new InvocationCallback<String>() {
@@ -399,10 +399,10 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public PartitionedTopicStats getPartitionedStats(String destination, boolean perPartition)
+ public PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition)
throws PulsarAdminException {
try {
- return getPartitionedStatsAsync(destination, perPartition).get();
+ return getPartitionedStatsAsync(topic, perPartition).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -412,9 +412,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String destination,
+ public CompletableFuture<PartitionedTopicStats> getPartitionedStatsAsync(String topic,
boolean perPartition) {
- DestinationName ds = validateTopic(destination);
+ TopicName ds = validateTopic(topic);
final CompletableFuture<PartitionedTopicStats> future = new CompletableFuture<>();
asyncGetRequest(
persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("partitioned-stats"),
@@ -437,9 +437,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void deleteSubscription(String destination, String subName) throws PulsarAdminException {
+ public void deleteSubscription(String topic, String subName) throws PulsarAdminException {
try {
- deleteSubscriptionAsync(destination, subName).get();
+ deleteSubscriptionAsync(topic, subName).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -449,17 +449,17 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> deleteSubscriptionAsync(String destination, String subName) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> deleteSubscriptionAsync(String topic, String subName) {
+ TopicName ds = validateTopic(topic);
String encodedSubName = Codec.encode(subName);
return asyncDeleteRequest(persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName())
.path("subscription").path(encodedSubName));
}
@Override
- public void skipAllMessages(String destination, String subName) throws PulsarAdminException {
+ public void skipAllMessages(String topic, String subName) throws PulsarAdminException {
try {
- skipAllMessagesAsync(destination, subName).get();
+ skipAllMessagesAsync(topic, subName).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -469,8 +469,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> skipAllMessagesAsync(String destination, String subName) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> skipAllMessagesAsync(String topic, String subName) {
+ TopicName ds = validateTopic(topic);
String encodedSubName = Codec.encode(subName);
return asyncPostRequest(
persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscription")
@@ -478,9 +478,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void skipMessages(String destination, String subName, long numMessages) throws PulsarAdminException {
+ public void skipMessages(String topic, String subName, long numMessages) throws PulsarAdminException {
try {
- skipMessagesAsync(destination, subName, numMessages).get();
+ skipMessagesAsync(topic, subName, numMessages).get();
} catch (ExecutionException e) {
throw (PulsarAdminException) e.getCause();
} catch (InterruptedException e) {
@@ -490,8 +490,8 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public CompletableFuture<Void> skipMessagesAsync(String destination, String subName, long numMessages) {
- DestinationName ds = validateTopic(destination);
+ public CompletableFuture<Void> skipMessagesAsync(String topic, String subName, long numMessages) {
+ TopicName ds = validateTopic(topic);
String encodedSubName = Codec.encode(subName);
return asyncPostRequest(
persistentTopics.path(ds.getNamespace()).path(ds.getEncodedLocalName()).path("subscription")
@@ -500,9 +500,9 @@ public class PersistentTopicsImpl extends BaseResource implements PersistentTopi
}
@Override
- public void expireMessages(String destination, String subName, long expireTimeInSeconds) throws PulsarAdminException {
+ public void expireMessages(String topic, String subName, long expireTimeInSeconds) throws PulsarAdminException {
try {
- expireMessagesAsync(destination, subName, expireTimeInSeconds).get();
+ expireMessagesAsync(topic, subName, expireTimeInSeconds).get();
} catch (ExecutionException e) {
... 2847 lines suppressed ...
--
To stop receiving notification emails like this one, please contact
mmerli@apache.org.