You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/06/14 16:37:38 UTC

[GitHub] [pulsar] andrasbeni opened a new pull request, #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

andrasbeni opened a new pull request, #16062:
URL: https://github.com/apache/pulsar/pull/16062

   Fixes #14505
   
   ### Motivation
   
   When using a pattern-based subscription, clients poll for new topics that
   match the pattern. It can happen frequently that the first messages produced
   to the topic will be consumed with a significant delay because the client
   has not got information about the topic for a while.  
   
   This commit introduces topic list watchers. By using these objects
   clients act as observers of the creation or deletion of topics closer to
   real-time. This reduces latency in consuming the first messages
   published to a topic when using a pattern-based subscription.
   
   ### Modifications
   
   - New commands were added to the binary protocol to enable registering
    and deregistering watchers.
   - Pattern-based consumers create TopicListWatcher objects if the broker
     supports this feature. Otherwise, they fall back to polling only.
   - The watchers use ConnectionHandler to obtain a connection to a broker.
   - Once connected, watchers register and wait for updates.
   - ServerCnx uses the newly created TopicListService to manage watchers.
   - TopicListService listens to metadata notifications and sends updates.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   This change added tests 
   
   ### Does this pull request potentially affect one of the following parts:
   
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API: no
     - The schema: no
     - The default values of configurations: no
     - The wire protocol: yes
        - Adds new commands and a new optional field to an existing command
     - The rest endpoints: no
     - The admin cli options: no
     - Anything that affects deployment: no
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [x] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [ ] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] merlimat commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
merlimat commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r907863444


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java:
##########
@@ -110,4 +123,34 @@ public CompletableFuture<Void> clearTenantPersistence(String tenant) {
                     }
                 });
     }
+
+    void handleNotification(Notification notification) {
+        if (notification.getPath().startsWith(MANAGED_LEDGER_PATH)
+                && EnumSet.of(NotificationType.Created, NotificationType.Deleted).contains(notification.getType())) {
+            for (Map.Entry<BiConsumer<String, NotificationType>, Pattern> entry :
+                    new HashMap<>(topicListeners).entrySet()) {
+                Matcher matcher = entry.getValue().matcher(notification.getPath());
+                if (matcher.matches()) {
+                    TopicName topicName = TopicName.get(
+                            matcher.group(2), NamespaceName.get(matcher.group(1)), matcher.group(3));
+                    entry.getKey().accept(topicName.toString(), notification.getType());
+                }
+            }
+        }
+    }
+
+    Pattern namespaceNameToTopicNamePattern(NamespaceName namespaceName) {

Review Comment:
   In `TopicName` we already have `getPersistenceName()` which is kind of similar. On one hand I agree that it's not directly related to the logical name, though it was a way to ensure all the naming related ops are in a single place with no duplicated approaches.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] andrasbeni commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
andrasbeni commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r902834448


##########
pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java:
##########
@@ -1474,6 +1477,55 @@ public static ByteBuf newEndTxnOnSubscriptionResponse(long requestId, long txnId
         return serializeWithSize(cmd);
     }
 
+    public static BaseCommand newWatchTopicList(
+            long requestId, long watcherId, String namespace, String topicsPattern, String topicsHash) {
+        BaseCommand cmd = localCmd(Type.WATCH_TOPIC_LIST);
+        cmd.setWatchTopicList()
+                .setRequestId(requestId)
+                .setNamespace(namespace)

Review Comment:
   Yes, that feature would make sense to me. However, the linked pull request has seen no progress for almost two years,  so if you don't mind I would like to keep the namespace parameter, for now, to simplify detecting when a metadata store notification is related to topics that match. Otherwise, it is more difficult to distinguish between topics in v1 namespaces and subscriptions in v2 namespaces. Or would you have a simple solution to that?  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] merlimat merged pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
merlimat merged PR #16062:
URL: https://github.com/apache/pulsar/pull/16062


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] andrasbeni commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
andrasbeni commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r902833402


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1036,6 +1053,62 @@ private TransactionBufferHandler checkAndGetTransactionBufferHandler() {
         return transactionBufferHandler;
     }
 
+    public CompletableFuture<CommandWatchTopicListSuccess> newWatchTopicList(
+            BaseCommand commandWatchTopicList, long requestId) {
+        if (!supportsTopicWatchers) {
+            return FutureUtil.failedFuture(
+                    new PulsarClientException.NotAllowedException(
+                            "Broker does not allow broker side pattern evaluation."));
+        }
+        return sendRequestAndHandleTimeout(Commands.serializeWithSize(commandWatchTopicList), requestId,
+                RequestType.Command, true);
+    }
+
+    public CompletableFuture<CommandWatchTopicListSuccess> newUnwatchTopicList(
+            BaseCommand commandUnwatchTopicList, long requestId) {
+        if (!supportsTopicWatchers) {
+            return FutureUtil.failedFuture(
+                    new PulsarClientException.NotAllowedException(
+                            "Broker does not allow broker side pattern evaluation."));
+        }
+        return sendRequestAndHandleTimeout(Commands.serializeWithSize(commandUnwatchTopicList), requestId,
+                RequestType.Command, true);
+    }

Review Comment:
   Nice catch. I am removing it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r897882350


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java:
##########
@@ -119,9 +119,11 @@ public enum State {
     // These sets are updated from multiple threads, so they require a threadsafe data structure
     private final Set<ProducerBase<?>> producers = Collections.newSetFromMap(new ConcurrentHashMap<>());
     private final Set<ConsumerBase<?>> consumers = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    private final Set<TopicListWatcher> topicListWatchers = Collections.newSetFromMap(new ConcurrentHashMap<>());

Review Comment:
   Looks like no one add watchers to this Set?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -1036,6 +1053,62 @@ private TransactionBufferHandler checkAndGetTransactionBufferHandler() {
         return transactionBufferHandler;
     }
 
+    public CompletableFuture<CommandWatchTopicListSuccess> newWatchTopicList(
+            BaseCommand commandWatchTopicList, long requestId) {
+        if (!supportsTopicWatchers) {
+            return FutureUtil.failedFuture(
+                    new PulsarClientException.NotAllowedException(
+                            "Broker does not allow broker side pattern evaluation."));
+        }
+        return sendRequestAndHandleTimeout(Commands.serializeWithSize(commandWatchTopicList), requestId,
+                RequestType.Command, true);
+    }
+
+    public CompletableFuture<CommandWatchTopicListSuccess> newUnwatchTopicList(
+            BaseCommand commandUnwatchTopicList, long requestId) {
+        if (!supportsTopicWatchers) {
+            return FutureUtil.failedFuture(
+                    new PulsarClientException.NotAllowedException(
+                            "Broker does not allow broker side pattern evaluation."));
+        }
+        return sendRequestAndHandleTimeout(Commands.serializeWithSize(commandUnwatchTopicList), requestId,
+                RequestType.Command, true);
+    }

Review Comment:
   Looks this method is unused. The TopicListWatcher uses `Commands.newUnwatchTopicList` directly.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java:
##########
@@ -0,0 +1,254 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.function.BiConsumer;
+import java.util.regex.Pattern;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.namespace.NamespaceService;
+import org.apache.pulsar.broker.resources.TopicResources;
+import org.apache.pulsar.common.api.proto.CommandUnwatchTopicList;
+import org.apache.pulsar.common.api.proto.CommandWatchTopicList;
+import org.apache.pulsar.common.api.proto.ServerError;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.topics.TopicList;
+import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap;
+import org.apache.pulsar.metadata.api.NotificationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TopicListService {
+
+
+    public static class TopicListWatcher implements BiConsumer<String, NotificationType> {
+
+        private final List<String> matchingTopics;
+        private final TopicListService topicListService;
+        private final long id;
+        private final Pattern topicsPattern;
+
+        public TopicListWatcher(TopicListService topicListService, long id,
+                                Pattern topicsPattern, List<String> topics) {
+            this.topicListService = topicListService;
+            this.id = id;
+            this.topicsPattern = topicsPattern;
+            this.matchingTopics = TopicList.filterTopics(topics, topicsPattern);
+        }
+
+        public List<String> getMatchingTopics() {
+            return matchingTopics;
+        }
+
+        @Override
+        public void accept(String topicName, NotificationType notificationType) {
+            if (topicsPattern.matcher(topicName).matches()) {
+                List<String> newTopics;
+                List<String> deletedTopics;
+                if (notificationType == NotificationType.Deleted) {
+                    newTopics = Collections.emptyList();
+                    deletedTopics = Collections.singletonList(topicName);
+                    matchingTopics.remove(topicName);
+                } else {
+                    deletedTopics = Collections.emptyList();
+                    newTopics = Collections.singletonList(topicName);
+                    matchingTopics.add(topicName);
+                }
+                String hash = TopicList.calculateHash(matchingTopics);
+                topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics);
+            }
+        }
+    }
+
+
+    private static final Logger log = LoggerFactory.getLogger(TopicListService.class);
+
+    private final NamespaceService namespaceService;
+    private final TopicResources topicResources;
+    private final ServerCnx connection;
+    private final boolean enableSubscriptionPatternEvaluation;
+    private final int maxSubscriptionPatternLength;
+    private final ConcurrentLongHashMap<CompletableFuture<TopicListWatcher>> watchers;
+
+
+    public TopicListService(PulsarService pulsar, ServerCnx connection,
+                            boolean enableSubscriptionPatternEvaluation, int maxSubscriptionPatternLength) {
+        this.namespaceService = pulsar.getNamespaceService();
+        this.connection = connection;
+        this.enableSubscriptionPatternEvaluation = enableSubscriptionPatternEvaluation;
+        this.maxSubscriptionPatternLength = maxSubscriptionPatternLength;
+        this.watchers = ConcurrentLongHashMap.<CompletableFuture<TopicListWatcher>>newBuilder()
+                .expectedItems(8)
+                .concurrencyLevel(1)
+                .build();
+        this.topicResources = pulsar.getPulsarResources().getTopicResources();
+    }
+
+    public void inactivate() {
+        for (Long watcherId : new HashSet<>(watchers.keys())) {
+            deleteTopicListWatcher(watcherId);
+        }
+    }
+
+    public void handleWatchTopicList(CommandWatchTopicList commandWatchTopicList, Semaphore lookupSemaphore) {
+        String namespace = commandWatchTopicList.getNamespace();
+        NamespaceName namespaceName = NamespaceName.get(namespace);
+        long watcherId = commandWatchTopicList.getWatcherId();
+        long requestId = commandWatchTopicList.getRequestId();
+
+        Pattern topicsPattern = Pattern.compile(commandWatchTopicList.hasTopicsPattern()
+                ? commandWatchTopicList.getTopicsPattern() : TopicList.ALL_TOPICS_PATTERN);
+        if (!enableSubscriptionPatternEvaluation || topicsPattern.pattern().length() > maxSubscriptionPatternLength) {
+            String msg = "Unable to create topic list watcher: ";
+            if (!enableSubscriptionPatternEvaluation) {
+                msg += "Evaluating subscription patterns is disabled.";
+            } else {
+                msg += "Pattern longer than maximum: " + maxSubscriptionPatternLength;
+            }
+            log.warn("[{}] {} on namespace {}", connection.getRemoteAddress(), msg, namespaceName);
+            connection.getCommandSender().sendErrorResponse(requestId, ServerError.NotAllowedError, msg);
+            lookupSemaphore.release();
+            return;
+        }
+        CompletableFuture<TopicListWatcher> watcherFuture = new CompletableFuture<>();
+        CompletableFuture<TopicListWatcher> existingWatcherFuture = watchers.putIfAbsent(watcherId, watcherFuture);
+
+        if (existingWatcherFuture != null) {
+            if (existingWatcherFuture.isDone() && !existingWatcherFuture.isCompletedExceptionally()) {
+                TopicListWatcher watcher = existingWatcherFuture.getNow(null);
+                log.info("[{}] Watcher with the same id is already created:"
+                                + " watcherId={}, watcher={}",
+                        connection.getRemoteAddress(), watcherId, watcher);
+                watcherFuture = existingWatcherFuture;
+            } else {
+                // There was an early request to create a watcher with the same watcherId. This can happen when
+                // client timeout is lower the broker timeouts. We need to wait until the previous watcher
+                // creation request either completes or fails.
+                log.warn("[{}] Watcher with id is already present on the connection,"
+                        + " consumerId={}", connection.getRemoteAddress(), watcherId);
+                ServerError error;
+                if (!existingWatcherFuture.isDone()) {
+                    error = ServerError.ServiceNotReady;
+                } else {
+                    error = ServerError.UnknownError;
+                    watchers.remove(watcherId, existingWatcherFuture);
+                }
+                connection.getCommandSender().sendErrorResponse(requestId, error,
+                        "Topic list watcher is already present on the connection");
+                lookupSemaphore.release();
+                return;
+            }
+        } else {
+            initializeTopicsListWatcher(watcherFuture, namespaceName, watcherId, topicsPattern);
+        }
+
+
+        watcherFuture.thenAccept(watcher -> {
+                    List<String> topicList = watcher.getMatchingTopics();
+                    String hash = TopicList.calculateHash(topicList);
+                    if (commandWatchTopicList.hasTopicsHash()
+                        && hash.equals(commandWatchTopicList.getTopicsHash())) {
+                        topicList = Collections.emptyList();
+                    }
+                    if (log.isDebugEnabled()) {
+                        log.debug(
+                                "[{}] Received WatchTopicList for namespace [//{}] by {}",
+                                connection.getRemoteAddress(), namespace, requestId);
+                    }
+                    connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList);
+                    lookupSemaphore.release();
+                })
+                .exceptionally(ex -> {
+                    log.warn("[{}] Error WatchTopicList for namespace [//{}] by {}",
+                            connection.getRemoteAddress(), namespace, requestId);
+                    connection.getCommandSender().sendErrorResponse(requestId,
+                            BrokerServiceException.getClientErrorCode(
+                                    new BrokerServiceException.ServerMetadataException(ex)), ex.getMessage());
+                    lookupSemaphore.release();
+                    return null;
+                });
+    }
+
+
+    public void initializeTopicsListWatcher(CompletableFuture<TopicListWatcher> watcherFuture,
+            NamespaceName namespace, long watcherId, Pattern topicsPattern) {
+        namespaceService.getListOfPersistentTopics(namespace).
+                thenApply(topics -> {
+                    TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, topics);
+                    topicResources.registerPersistentTopicListener(namespace, watcher);
+                    return watcher;
+                }).
+                whenComplete((watcher, exception) -> {
+                    if (exception != null) {
+                        watcherFuture.completeExceptionally(exception);
+                    } else {
+                        watcherFuture.complete(watcher);
+                    }
+                });
+    }
+
+
+    public void handleUnwatchTopicList(CommandUnwatchTopicList commandUnwatchTopicList) {
+        long requestId = commandUnwatchTopicList.getRequestId();
+        long watcherId = commandUnwatchTopicList.getWatcherId();
+        deleteTopicListWatcher(watcherId);
+        connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, null, null);
+    }
+
+    public void deleteTopicListWatcher(Long watcherId) {
+        CompletableFuture<TopicListWatcher> watcherFuture = watchers.get(watcherId);
+        if (watcherFuture == null) {
+            log.info("[{}] TopicListWatcher was not registered on the connection: {}",
+                    watcherId, connection.getRemoteAddress());
+            return;
+        }
+
+        if (!watcherFuture.isDone() && watcherFuture
+                .completeExceptionally(new IllegalStateException("Closed watcher before creation was complete"))) {
+            // We have received a request to close the watcher before it was actually completed, we have marked the
+            // watcher future as failed and we can tell the client the close operation was successful. When the actual
+            // create operation will complete, the new watcher will be discarded.
+            log.info("[{}] Closed watcher before its creation was completed. watcherId={}",
+                    connection.getRemoteAddress(), watcherId);
+            return;

Review Comment:
   Any reason that doesn't remove from the watchers map if reach here?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java:
##########
@@ -0,0 +1,287 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl;
+
+import io.netty.channel.ChannelHandlerContext;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.api.proto.BaseCommand;
+import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TopicListWatcher extends HandlerState implements ConnectionHandler.Connection {
+
+    private static final Logger log = LoggerFactory.getLogger(TopicListWatcher.class);
+
+    private static final AtomicLongFieldUpdater<TopicListWatcher> CREATE_WATCHER_DEADLINE_UPDATER =
+            AtomicLongFieldUpdater
+                    .newUpdater(TopicListWatcher.class, "createWatcherDeadline");
+
+    private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener;
+    private final String name;
+    private final ConnectionHandler connectionHandler;
+    private final Pattern topicsPattern;
+    private final long watcherId;
+    private volatile long createWatcherDeadline = 0;
+    private final NamespaceName namespace;
+    // TODO maintain the value based on updates from broker and warn the user if inconsistent with hash from polling
+    private String topicsHash;
+    private final CompletableFuture<TopicListWatcher> watcherFuture;
+
+    private final List<Throwable> previousExceptions = new CopyOnWriteArrayList<>();
+    private final AtomicReference<ClientCnx> clientCnxUsedForWatcherRegistration = new AtomicReference<>();
+
+
+    public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener,
+                            PulsarClientImpl client, Pattern topicsPattern, long watcherId,
+                            NamespaceName namespace, String topicsHash,
+                            CompletableFuture<TopicListWatcher> watcherFuture) {
+        super(client, null);
+        this.topicsChangeListener = topicsChangeListener;
+        this.name = "Watcher(" + topicsPattern + ")";
+        this.connectionHandler = new ConnectionHandler(this,
+                new BackoffBuilder()
+                        .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(),
+                                TimeUnit.NANOSECONDS)
+                        .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
+                        .setMandatoryStop(0, TimeUnit.MILLISECONDS)
+                        .create(),
+                this);
+        this.topicsPattern = topicsPattern;
+        this.watcherId = watcherId;
+        this.namespace = namespace;
+        this.topicsHash = topicsHash;
+        this.watcherFuture = watcherFuture;
+
+        connectionHandler.grabCnx();
+    }
+
+    @Override
+    public void connectionFailed(PulsarClientException exception) {
+        boolean nonRetriableError = !PulsarClientException.isRetriableError(exception);
+        if (nonRetriableError) {
+            exception.setPreviousExceptions(previousExceptions);
+            if (watcherFuture.completeExceptionally(exception)) {
+                setState(State.Failed);
+                log.info("[{}] Watcher creation failed for {} with non-retriable error {}",
+                        topic, name, exception);
+                deregisterFromClientCnx();
+                client.cleanupTopicListWatcher(this);
+            }
+        } else {
+            previousExceptions.add(exception);
+        }
+    }
+
+    @Override
+    public void connectionOpened(ClientCnx cnx) {
+        previousExceptions.clear();
+
+        if (getState() == State.Closing || getState() == State.Closed) {
+            setState(State.Closed);
+            deregisterFromClientCnx();
+            client.cleanupTopicListWatcher(this);
+            return;
+        }
+
+        log.info("[{}][{}] Creating topic list watcher on cnx {}, watcherId {}",
+                topic, getHandlerName(), cnx.ctx().channel(), watcherId);
+
+        long requestId = client.newRequestId();
+
+        CREATE_WATCHER_DEADLINE_UPDATER
+                .compareAndSet(this, 0L, System.currentTimeMillis()
+                        + client.getConfiguration().getOperationTimeoutMs());
+
+        // synchronized this, because redeliverUnAckMessage eliminate the epoch inconsistency between them
+        synchronized (this) {
+            setClientCnx(cnx);
+            BaseCommand watchRequest = Commands.newWatchTopicList(requestId, watcherId, namespace.toString(),
+                            topicsPattern.pattern(), topicsHash);
+
+            cnx.newWatchTopicList(watchRequest, requestId)
+
+                    .thenAccept(response -> {
+                        synchronized (TopicListWatcher.this) {
+                            if (!changeToReadyState()) {
+                                // Watcher was closed while reconnecting, close the connection to make sure the broker
+                                // drops the watcher on its side
+                                setState(State.Closed);
+                                deregisterFromClientCnx();
+                                client.cleanupTopicListWatcher(this);
+
+                                cnx.channel().close();
+                                return;
+                            }
+                        }
+
+                        this.connectionHandler.resetBackoff();
+
+                        watcherFuture.complete(this);
+
+                    }).exceptionally((e) -> {
+                        deregisterFromClientCnx();
+                        if (getState() == State.Closing || getState() == State.Closed) {
+                            // Watcher was closed while reconnecting, close the connection to make sure the broker
+                            // drops the watcher on its side
+                            cnx.channel().close();
+                            return null;
+                        }
+                        log.warn("[{}][{}] Failed to subscribe to topic on {}", topic,
+                                getHandlerName(), cnx.channel().remoteAddress());
+
+                        if (e.getCause() instanceof PulsarClientException
+                                && PulsarClientException.isRetriableError(e.getCause())
+                                && System.currentTimeMillis()
+                                    < CREATE_WATCHER_DEADLINE_UPDATER.get(TopicListWatcher.this)) {
+                            reconnectLater(e.getCause());
+                        } else if (!watcherFuture.isDone()) {
+                            // unable to create new watcher, fail operation
+                            setState(State.Failed);
+                            watcherFuture.completeExceptionally(
+                                    PulsarClientException.wrap(e, String.format("Failed to create topic list watcher %s"
+                                                    + "when connecting to the broker", getHandlerName())));
+                            client.cleanupTopicListWatcher(this);

Review Comment:
   Should also cleanup the watcher from the `ClientCnx.topicListWatchers` since line:123 added the watcher to the `ClientCnx.topicListWatchers`



##########
pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java:
##########
@@ -1474,6 +1477,55 @@ public static ByteBuf newEndTxnOnSubscriptionResponse(long requestId, long txnId
         return serializeWithSize(cmd);
     }
 
+    public static BaseCommand newWatchTopicList(
+            long requestId, long watcherId, String namespace, String topicsPattern, String topicsHash) {
+        BaseCommand cmd = localCmd(Type.WATCH_TOPIC_LIST);
+        cmd.setWatchTopicList()
+                .setRequestId(requestId)
+                .setNamespace(namespace)

Review Comment:
   Can we make the namespace optional? Since we might support pattern subscribe without namespace/tenant. A related PR https://github.com/apache/pulsar/pull/7855



##########
pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java:
##########
@@ -190,4 +196,106 @@ public void testHandleCloseProducer() {
 
         eventLoop.shutdownGracefully();
     }
+
+    @Test
+    public void testNoWatchersWhenNoServerSupport() {
+        withConnection("testNoWatchersWhenNoServerSupport", cnx -> {
+            cnx.handleConnected(new CommandConnected()
+                    .setServerVersion("Some old Server")
+                    .setProtocolVersion(1));
+
+            CompletableFuture<CommandWatchTopicListSuccess> result =
+                    cnx.newWatchTopicList(Commands.newWatchTopicList(7, 5, "tenant/ns",
+                            ".*", null), 7);
+            assertTrue(result.isCompletedExceptionally());

Review Comment:
   Please also check the watcher is cleaned up if server does not support.



##########
pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java:
##########
@@ -1474,6 +1477,55 @@ public static ByteBuf newEndTxnOnSubscriptionResponse(long requestId, long txnId
         return serializeWithSize(cmd);
     }
 
+    public static BaseCommand newWatchTopicList(
+            long requestId, long watcherId, String namespace, String topicsPattern, String topicsHash) {
+        BaseCommand cmd = localCmd(Type.WATCH_TOPIC_LIST);
+        cmd.setWatchTopicList()
+                .setRequestId(requestId)
+                .setNamespace(namespace)

Review Comment:
   Or maybe better to have tenant and namespace, just keep them optional.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java:
##########
@@ -0,0 +1,254 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.service;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.function.BiConsumer;
+import java.util.regex.Pattern;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.namespace.NamespaceService;
+import org.apache.pulsar.broker.resources.TopicResources;
+import org.apache.pulsar.common.api.proto.CommandUnwatchTopicList;
+import org.apache.pulsar.common.api.proto.CommandWatchTopicList;
+import org.apache.pulsar.common.api.proto.ServerError;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.topics.TopicList;
+import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap;
+import org.apache.pulsar.metadata.api.NotificationType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TopicListService {
+
+
+    public static class TopicListWatcher implements BiConsumer<String, NotificationType> {
+
+        private final List<String> matchingTopics;
+        private final TopicListService topicListService;
+        private final long id;
+        private final Pattern topicsPattern;
+
+        public TopicListWatcher(TopicListService topicListService, long id,
+                                Pattern topicsPattern, List<String> topics) {
+            this.topicListService = topicListService;
+            this.id = id;
+            this.topicsPattern = topicsPattern;
+            this.matchingTopics = TopicList.filterTopics(topics, topicsPattern);
+        }
+
+        public List<String> getMatchingTopics() {
+            return matchingTopics;
+        }
+
+        @Override
+        public void accept(String topicName, NotificationType notificationType) {
+            if (topicsPattern.matcher(topicName).matches()) {
+                List<String> newTopics;
+                List<String> deletedTopics;
+                if (notificationType == NotificationType.Deleted) {
+                    newTopics = Collections.emptyList();
+                    deletedTopics = Collections.singletonList(topicName);
+                    matchingTopics.remove(topicName);
+                } else {
+                    deletedTopics = Collections.emptyList();
+                    newTopics = Collections.singletonList(topicName);
+                    matchingTopics.add(topicName);
+                }
+                String hash = TopicList.calculateHash(matchingTopics);
+                topicListService.sendTopicListUpdate(id, hash, deletedTopics, newTopics);
+            }
+        }
+    }
+
+
+    private static final Logger log = LoggerFactory.getLogger(TopicListService.class);
+
+    private final NamespaceService namespaceService;
+    private final TopicResources topicResources;
+    private final ServerCnx connection;
+    private final boolean enableSubscriptionPatternEvaluation;
+    private final int maxSubscriptionPatternLength;
+    private final ConcurrentLongHashMap<CompletableFuture<TopicListWatcher>> watchers;
+
+
+    public TopicListService(PulsarService pulsar, ServerCnx connection,
+                            boolean enableSubscriptionPatternEvaluation, int maxSubscriptionPatternLength) {
+        this.namespaceService = pulsar.getNamespaceService();
+        this.connection = connection;
+        this.enableSubscriptionPatternEvaluation = enableSubscriptionPatternEvaluation;
+        this.maxSubscriptionPatternLength = maxSubscriptionPatternLength;
+        this.watchers = ConcurrentLongHashMap.<CompletableFuture<TopicListWatcher>>newBuilder()
+                .expectedItems(8)
+                .concurrencyLevel(1)
+                .build();
+        this.topicResources = pulsar.getPulsarResources().getTopicResources();
+    }
+
+    public void inactivate() {
+        for (Long watcherId : new HashSet<>(watchers.keys())) {
+            deleteTopicListWatcher(watcherId);
+        }
+    }
+
+    public void handleWatchTopicList(CommandWatchTopicList commandWatchTopicList, Semaphore lookupSemaphore) {
+        String namespace = commandWatchTopicList.getNamespace();
+        NamespaceName namespaceName = NamespaceName.get(namespace);
+        long watcherId = commandWatchTopicList.getWatcherId();
+        long requestId = commandWatchTopicList.getRequestId();
+
+        Pattern topicsPattern = Pattern.compile(commandWatchTopicList.hasTopicsPattern()
+                ? commandWatchTopicList.getTopicsPattern() : TopicList.ALL_TOPICS_PATTERN);
+        if (!enableSubscriptionPatternEvaluation || topicsPattern.pattern().length() > maxSubscriptionPatternLength) {
+            String msg = "Unable to create topic list watcher: ";
+            if (!enableSubscriptionPatternEvaluation) {
+                msg += "Evaluating subscription patterns is disabled.";
+            } else {
+                msg += "Pattern longer than maximum: " + maxSubscriptionPatternLength;
+            }
+            log.warn("[{}] {} on namespace {}", connection.getRemoteAddress(), msg, namespaceName);
+            connection.getCommandSender().sendErrorResponse(requestId, ServerError.NotAllowedError, msg);
+            lookupSemaphore.release();
+            return;
+        }
+        CompletableFuture<TopicListWatcher> watcherFuture = new CompletableFuture<>();
+        CompletableFuture<TopicListWatcher> existingWatcherFuture = watchers.putIfAbsent(watcherId, watcherFuture);
+
+        if (existingWatcherFuture != null) {
+            if (existingWatcherFuture.isDone() && !existingWatcherFuture.isCompletedExceptionally()) {
+                TopicListWatcher watcher = existingWatcherFuture.getNow(null);
+                log.info("[{}] Watcher with the same id is already created:"
+                                + " watcherId={}, watcher={}",
+                        connection.getRemoteAddress(), watcherId, watcher);
+                watcherFuture = existingWatcherFuture;
+            } else {
+                // There was an early request to create a watcher with the same watcherId. This can happen when
+                // client timeout is lower the broker timeouts. We need to wait until the previous watcher
+                // creation request either completes or fails.
+                log.warn("[{}] Watcher with id is already present on the connection,"
+                        + " consumerId={}", connection.getRemoteAddress(), watcherId);
+                ServerError error;
+                if (!existingWatcherFuture.isDone()) {
+                    error = ServerError.ServiceNotReady;
+                } else {
+                    error = ServerError.UnknownError;
+                    watchers.remove(watcherId, existingWatcherFuture);
+                }
+                connection.getCommandSender().sendErrorResponse(requestId, error,
+                        "Topic list watcher is already present on the connection");
+                lookupSemaphore.release();
+                return;
+            }
+        } else {
+            initializeTopicsListWatcher(watcherFuture, namespaceName, watcherId, topicsPattern);
+        }
+
+
+        watcherFuture.thenAccept(watcher -> {
+                    List<String> topicList = watcher.getMatchingTopics();
+                    String hash = TopicList.calculateHash(topicList);
+                    if (commandWatchTopicList.hasTopicsHash()
+                        && hash.equals(commandWatchTopicList.getTopicsHash())) {
+                        topicList = Collections.emptyList();
+                    }
+                    if (log.isDebugEnabled()) {
+                        log.debug(
+                                "[{}] Received WatchTopicList for namespace [//{}] by {}",
+                                connection.getRemoteAddress(), namespace, requestId);
+                    }
+                    connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList);
+                    lookupSemaphore.release();
+                })
+                .exceptionally(ex -> {
+                    log.warn("[{}] Error WatchTopicList for namespace [//{}] by {}",
+                            connection.getRemoteAddress(), namespace, requestId);
+                    connection.getCommandSender().sendErrorResponse(requestId,
+                            BrokerServiceException.getClientErrorCode(
+                                    new BrokerServiceException.ServerMetadataException(ex)), ex.getMessage());
+                    lookupSemaphore.release();
+                    return null;
+                });
+    }
+
+
+    public void initializeTopicsListWatcher(CompletableFuture<TopicListWatcher> watcherFuture,
+            NamespaceName namespace, long watcherId, Pattern topicsPattern) {
+        namespaceService.getListOfPersistentTopics(namespace).
+                thenApply(topics -> {
+                    TopicListWatcher watcher = new TopicListWatcher(this, watcherId, topicsPattern, topics);
+                    topicResources.registerPersistentTopicListener(namespace, watcher);
+                    return watcher;
+                }).
+                whenComplete((watcher, exception) -> {
+                    if (exception != null) {
+                        watcherFuture.completeExceptionally(exception);
+                    } else {
+                        watcherFuture.complete(watcher);
+                    }
+                });
+    }
+
+
+    public void handleUnwatchTopicList(CommandUnwatchTopicList commandUnwatchTopicList) {
+        long requestId = commandUnwatchTopicList.getRequestId();
+        long watcherId = commandUnwatchTopicList.getWatcherId();
+        deleteTopicListWatcher(watcherId);
+        connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, null, null);
+    }
+
+    public void deleteTopicListWatcher(Long watcherId) {
+        CompletableFuture<TopicListWatcher> watcherFuture = watchers.get(watcherId);
+        if (watcherFuture == null) {
+            log.info("[{}] TopicListWatcher was not registered on the connection: {}",
+                    watcherId, connection.getRemoteAddress());
+            return;
+        }
+
+        if (!watcherFuture.isDone() && watcherFuture
+                .completeExceptionally(new IllegalStateException("Closed watcher before creation was complete"))) {
+            // We have received a request to close the watcher before it was actually completed, we have marked the
+            // watcher future as failed and we can tell the client the close operation was successful. When the actual
+            // create operation will complete, the new watcher will be discarded.
+            log.info("[{}] Closed watcher before its creation was completed. watcherId={}",
+                    connection.getRemoteAddress(), watcherId);
+            return;
+        }
+
+        if (watcherFuture.isCompletedExceptionally()) {
+            log.info("[{}] Closed watcher that already failed to be created. watcherId={}",
+                    connection.getRemoteAddress(), watcherId);
+            return;

Review Comment:
   Same as the above comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] andrasbeni commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
andrasbeni commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r906179748


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java:
##########
@@ -110,4 +123,34 @@ public CompletableFuture<Void> clearTenantPersistence(String tenant) {
                     }
                 });
     }
+
+    void handleNotification(Notification notification) {
+        if (notification.getPath().startsWith(MANAGED_LEDGER_PATH)
+                && EnumSet.of(NotificationType.Created, NotificationType.Deleted).contains(notification.getType())) {
+            for (Map.Entry<BiConsumer<String, NotificationType>, Pattern> entry :
+                    new HashMap<>(topicListeners).entrySet()) {
+                Matcher matcher = entry.getValue().matcher(notification.getPath());
+                if (matcher.matches()) {
+                    TopicName topicName = TopicName.get(
+                            matcher.group(2), NamespaceName.get(matcher.group(1)), matcher.group(3));
+                    entry.getKey().accept(topicName.toString(), notification.getType());
+                }
+            }
+        }
+    }
+
+    Pattern namespaceNameToTopicNamePattern(NamespaceName namespaceName) {

Review Comment:
   That would improve performance indeed. The only problem I'd have with it is that we'd be referring to a relatively low-level detail (the managed ledger path) in a very general and high-level class. To me, it feels like breaking encapsulation/abstraction.
   What do you think about caching the patterns in this class instead? Something along the lines of `NamespaceName.cache`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] andrasbeni commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
andrasbeni commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r902833151


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java:
##########
@@ -119,9 +119,11 @@ public enum State {
     // These sets are updated from multiple threads, so they require a threadsafe data structure
     private final Set<ProducerBase<?>> producers = Collections.newSetFromMap(new ConcurrentHashMap<>());
     private final Set<ConsumerBase<?>> consumers = Collections.newSetFromMap(new ConcurrentHashMap<>());
+    private final Set<TopicListWatcher> topicListWatchers = Collections.newSetFromMap(new ConcurrentHashMap<>());

Review Comment:
   You are right. This set is redundant to ClientCnx.topicListWatchers, so I'm going to remove this.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java:
##########
@@ -0,0 +1,287 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl;
+
+import io.netty.channel.ChannelHandlerContext;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.api.proto.BaseCommand;
+import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TopicListWatcher extends HandlerState implements ConnectionHandler.Connection {
+
+    private static final Logger log = LoggerFactory.getLogger(TopicListWatcher.class);
+
+    private static final AtomicLongFieldUpdater<TopicListWatcher> CREATE_WATCHER_DEADLINE_UPDATER =
+            AtomicLongFieldUpdater
+                    .newUpdater(TopicListWatcher.class, "createWatcherDeadline");
+
+    private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener;
+    private final String name;
+    private final ConnectionHandler connectionHandler;
+    private final Pattern topicsPattern;
+    private final long watcherId;
+    private volatile long createWatcherDeadline = 0;
+    private final NamespaceName namespace;
+    // TODO maintain the value based on updates from broker and warn the user if inconsistent with hash from polling
+    private String topicsHash;
+    private final CompletableFuture<TopicListWatcher> watcherFuture;
+
+    private final List<Throwable> previousExceptions = new CopyOnWriteArrayList<>();
+    private final AtomicReference<ClientCnx> clientCnxUsedForWatcherRegistration = new AtomicReference<>();
+
+
+    public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener,
+                            PulsarClientImpl client, Pattern topicsPattern, long watcherId,
+                            NamespaceName namespace, String topicsHash,
+                            CompletableFuture<TopicListWatcher> watcherFuture) {
+        super(client, null);
+        this.topicsChangeListener = topicsChangeListener;
+        this.name = "Watcher(" + topicsPattern + ")";
+        this.connectionHandler = new ConnectionHandler(this,
+                new BackoffBuilder()
+                        .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(),
+                                TimeUnit.NANOSECONDS)
+                        .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
+                        .setMandatoryStop(0, TimeUnit.MILLISECONDS)
+                        .create(),
+                this);
+        this.topicsPattern = topicsPattern;
+        this.watcherId = watcherId;
+        this.namespace = namespace;
+        this.topicsHash = topicsHash;
+        this.watcherFuture = watcherFuture;
+
+        connectionHandler.grabCnx();
+    }
+
+    @Override
+    public void connectionFailed(PulsarClientException exception) {
+        boolean nonRetriableError = !PulsarClientException.isRetriableError(exception);
+        if (nonRetriableError) {
+            exception.setPreviousExceptions(previousExceptions);
+            if (watcherFuture.completeExceptionally(exception)) {
+                setState(State.Failed);
+                log.info("[{}] Watcher creation failed for {} with non-retriable error {}",
+                        topic, name, exception);
+                deregisterFromClientCnx();
+                client.cleanupTopicListWatcher(this);
+            }
+        } else {
+            previousExceptions.add(exception);
+        }
+    }
+
+    @Override
+    public void connectionOpened(ClientCnx cnx) {
+        previousExceptions.clear();
+
+        if (getState() == State.Closing || getState() == State.Closed) {
+            setState(State.Closed);
+            deregisterFromClientCnx();
+            client.cleanupTopicListWatcher(this);
+            return;
+        }
+
+        log.info("[{}][{}] Creating topic list watcher on cnx {}, watcherId {}",
+                topic, getHandlerName(), cnx.ctx().channel(), watcherId);
+
+        long requestId = client.newRequestId();
+
+        CREATE_WATCHER_DEADLINE_UPDATER
+                .compareAndSet(this, 0L, System.currentTimeMillis()
+                        + client.getConfiguration().getOperationTimeoutMs());
+
+        // synchronized this, because redeliverUnAckMessage eliminate the epoch inconsistency between them
+        synchronized (this) {
+            setClientCnx(cnx);
+            BaseCommand watchRequest = Commands.newWatchTopicList(requestId, watcherId, namespace.toString(),
+                            topicsPattern.pattern(), topicsHash);
+
+            cnx.newWatchTopicList(watchRequest, requestId)
+
+                    .thenAccept(response -> {
+                        synchronized (TopicListWatcher.this) {
+                            if (!changeToReadyState()) {
+                                // Watcher was closed while reconnecting, close the connection to make sure the broker
+                                // drops the watcher on its side
+                                setState(State.Closed);
+                                deregisterFromClientCnx();
+                                client.cleanupTopicListWatcher(this);
+
+                                cnx.channel().close();
+                                return;
+                            }
+                        }
+
+                        this.connectionHandler.resetBackoff();
+
+                        watcherFuture.complete(this);
+
+                    }).exceptionally((e) -> {
+                        deregisterFromClientCnx();
+                        if (getState() == State.Closing || getState() == State.Closed) {
+                            // Watcher was closed while reconnecting, close the connection to make sure the broker
+                            // drops the watcher on its side
+                            cnx.channel().close();
+                            return null;
+                        }
+                        log.warn("[{}][{}] Failed to subscribe to topic on {}", topic,
+                                getHandlerName(), cnx.channel().remoteAddress());
+
+                        if (e.getCause() instanceof PulsarClientException
+                                && PulsarClientException.isRetriableError(e.getCause())
+                                && System.currentTimeMillis()
+                                    < CREATE_WATCHER_DEADLINE_UPDATER.get(TopicListWatcher.this)) {
+                            reconnectLater(e.getCause());
+                        } else if (!watcherFuture.isDone()) {
+                            // unable to create new watcher, fail operation
+                            setState(State.Failed);
+                            watcherFuture.completeExceptionally(
+                                    PulsarClientException.wrap(e, String.format("Failed to create topic list watcher %s"
+                                                    + "when connecting to the broker", getHandlerName())));
+                            client.cleanupTopicListWatcher(this);

Review Comment:
   `deregisterFromClientCnx()`, this `exceptionally` block's first call does call `ClientCnx.removeTopicListWatcher(long watcherId)`, so I think it should be fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] andrasbeni commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
andrasbeni commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r906179524


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2513,6 +2519,53 @@ protected void handleAddSubscriptionToTxn(CommandAddSubscriptionToTxn command) {
                 }));
     }
 
+    protected void handleCommandWatchTopicList(CommandWatchTopicList commandWatchTopicList) {
+        final long requestId = commandWatchTopicList.getRequestId();
+        final long watcherId = commandWatchTopicList.getWatcherId();
+        final NamespaceName namespaceName = NamespaceName.get(commandWatchTopicList.getNamespace());
+
+        final Semaphore lookupSemaphore = service.getLookupRequestSemaphore();
+        if (lookupSemaphore.tryAcquire()) {
+            if (invalidOriginalPrincipal(originalPrincipal)) {
+                final String msg = "Valid Proxy Client role should be provided for watchTopicListRequest ";
+                log.warn("[{}] {} with role {} and proxyClientAuthRole {} on namespace {}", remoteAddress, msg,
+                        authRole, originalPrincipal, namespaceName);
+                commandSender.sendErrorResponse(watcherId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return;
+            }
+            isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> {
+                if (isAuthorized) {
+                    topicListService.handleWatchTopicList(commandWatchTopicList, lookupSemaphore);
+                } else {
+                    final String msg = "Proxy Client is not authorized to watchTopicList";
+                    log.warn("[{}] {} with role {} on namespace {}", remoteAddress, msg, getPrincipal(), namespaceName);
+                    commandSender.sendErrorResponse(requestId, ServerError.AuthorizationError, msg);
+                    lookupSemaphore.release();
+                }
+                return null;
+            }).exceptionally(ex -> {
+                logNamespaceNameAuthException(remoteAddress, "watchTopicList", getPrincipal(),
+                        Optional.of(namespaceName), ex);
+                final String msg = "Exception occurred while trying to handle command WatchTopicList";
+                commandSender.sendErrorResponse(requestId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return null;
+            });
+        } else {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Failed WatchTopicList due to too many lookup-requests {}", remoteAddress,
+                        namespaceName);
+            }
+            commandSender.sendErrorResponse(requestId, ServerError.TooManyRequests,
+                    "Failed due to too many pending lookup requests");
+        }
+    }
+
+    protected void handleCommandUnwatchTopicList(CommandUnwatchTopicList commandUnwatchTopicList) {
+        topicListService.handleUnwatchTopicList(commandUnwatchTopicList);

Review Comment:
   We don't need to check authorization because executing this command only alters the state of `TopicListService`, which is in a 1:1 relation with `ServerCnx`. So it belongs to a single client, which was already checked when it created the watcher. (And if it did not have the authz to create one, it will be unable to delete anything)
   Similar to `handleCloseConsumer`.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2513,6 +2519,53 @@ protected void handleAddSubscriptionToTxn(CommandAddSubscriptionToTxn command) {
                 }));
     }
 
+    protected void handleCommandWatchTopicList(CommandWatchTopicList commandWatchTopicList) {
+        final long requestId = commandWatchTopicList.getRequestId();
+        final long watcherId = commandWatchTopicList.getWatcherId();
+        final NamespaceName namespaceName = NamespaceName.get(commandWatchTopicList.getNamespace());
+
+        final Semaphore lookupSemaphore = service.getLookupRequestSemaphore();
+        if (lookupSemaphore.tryAcquire()) {
+            if (invalidOriginalPrincipal(originalPrincipal)) {
+                final String msg = "Valid Proxy Client role should be provided for watchTopicListRequest ";
+                log.warn("[{}] {} with role {} and proxyClientAuthRole {} on namespace {}", remoteAddress, msg,
+                        authRole, originalPrincipal, namespaceName);
+                commandSender.sendErrorResponse(watcherId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return;
+            }
+            isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> {
+                if (isAuthorized) {
+                    topicListService.handleWatchTopicList(commandWatchTopicList, lookupSemaphore);

Review Comment:
   Thanks for pointing this out. I haven't thought of it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] merlimat commented on a diff in pull request #16062: [feat][broker] PIP-145: Notifications for faster topic discovery

Posted by GitBox <gi...@apache.org>.
merlimat commented on code in PR #16062:
URL: https://github.com/apache/pulsar/pull/16062#discussion_r905309249


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java:
##########
@@ -110,4 +123,34 @@ public CompletableFuture<Void> clearTenantPersistence(String tenant) {
                     }
                 });
     }
+
+    void handleNotification(Notification notification) {
+        if (notification.getPath().startsWith(MANAGED_LEDGER_PATH)
+                && EnumSet.of(NotificationType.Created, NotificationType.Deleted).contains(notification.getType())) {
+            for (Map.Entry<BiConsumer<String, NotificationType>, Pattern> entry :
+                    new HashMap<>(topicListeners).entrySet()) {
+                Matcher matcher = entry.getValue().matcher(notification.getPath());
+                if (matcher.matches()) {
+                    TopicName topicName = TopicName.get(
+                            matcher.group(2), NamespaceName.get(matcher.group(1)), matcher.group(3));
+                    entry.getKey().accept(topicName.toString(), notification.getType());
+                }
+            }
+        }
+    }
+
+    Pattern namespaceNameToTopicNamePattern(NamespaceName namespaceName) {

Review Comment:
   We could have this as part of `NamespaceName` where we can store the compiled pattern.



##########
pulsar-common/src/main/proto/PulsarApi.proto:
##########
@@ -987,6 +1017,11 @@ message BaseCommand {
         TC_CLIENT_CONNECT_REQUEST = 62;
         TC_CLIENT_CONNECT_RESPONSE = 63;
 
+        WATCH_TOPIC_LIST = 64;
+        WATCH_TOPIC_LIST_SUCCESS = 65;
+        WATCH_TOPIC_UPDATE = 66;
+        UNWATCH_TOPIC_LIST = 67;

Review Comment:
   nit: maybe we could rename to `WATCH_TOPIC_CLOSE` to have the same prefix



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2513,6 +2519,53 @@ protected void handleAddSubscriptionToTxn(CommandAddSubscriptionToTxn command) {
                 }));
     }
 
+    protected void handleCommandWatchTopicList(CommandWatchTopicList commandWatchTopicList) {
+        final long requestId = commandWatchTopicList.getRequestId();
+        final long watcherId = commandWatchTopicList.getWatcherId();
+        final NamespaceName namespaceName = NamespaceName.get(commandWatchTopicList.getNamespace());
+
+        final Semaphore lookupSemaphore = service.getLookupRequestSemaphore();
+        if (lookupSemaphore.tryAcquire()) {
+            if (invalidOriginalPrincipal(originalPrincipal)) {
+                final String msg = "Valid Proxy Client role should be provided for watchTopicListRequest ";
+                log.warn("[{}] {} with role {} and proxyClientAuthRole {} on namespace {}", remoteAddress, msg,
+                        authRole, originalPrincipal, namespaceName);
+                commandSender.sendErrorResponse(watcherId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return;
+            }
+            isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> {
+                if (isAuthorized) {
+                    topicListService.handleWatchTopicList(commandWatchTopicList, lookupSemaphore);

Review Comment:
   This method will potentially get called from a different thread (if the authz is not already cached), therefore we cannot pass on the `commandWatchTopicList` object because it will get reused when the io-thread is processing the next call. 
   
   We need to extract the fields that we are interested in into local variables.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2513,6 +2519,53 @@ protected void handleAddSubscriptionToTxn(CommandAddSubscriptionToTxn command) {
                 }));
     }
 
+    protected void handleCommandWatchTopicList(CommandWatchTopicList commandWatchTopicList) {
+        final long requestId = commandWatchTopicList.getRequestId();
+        final long watcherId = commandWatchTopicList.getWatcherId();
+        final NamespaceName namespaceName = NamespaceName.get(commandWatchTopicList.getNamespace());
+
+        final Semaphore lookupSemaphore = service.getLookupRequestSemaphore();
+        if (lookupSemaphore.tryAcquire()) {
+            if (invalidOriginalPrincipal(originalPrincipal)) {
+                final String msg = "Valid Proxy Client role should be provided for watchTopicListRequest ";
+                log.warn("[{}] {} with role {} and proxyClientAuthRole {} on namespace {}", remoteAddress, msg,
+                        authRole, originalPrincipal, namespaceName);
+                commandSender.sendErrorResponse(watcherId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return;
+            }
+            isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> {
+                if (isAuthorized) {
+                    topicListService.handleWatchTopicList(commandWatchTopicList, lookupSemaphore);
+                } else {
+                    final String msg = "Proxy Client is not authorized to watchTopicList";
+                    log.warn("[{}] {} with role {} on namespace {}", remoteAddress, msg, getPrincipal(), namespaceName);
+                    commandSender.sendErrorResponse(requestId, ServerError.AuthorizationError, msg);
+                    lookupSemaphore.release();
+                }
+                return null;
+            }).exceptionally(ex -> {
+                logNamespaceNameAuthException(remoteAddress, "watchTopicList", getPrincipal(),
+                        Optional.of(namespaceName), ex);
+                final String msg = "Exception occurred while trying to handle command WatchTopicList";
+                commandSender.sendErrorResponse(requestId, ServerError.AuthorizationError, msg);
+                lookupSemaphore.release();
+                return null;
+            });
+        } else {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Failed WatchTopicList due to too many lookup-requests {}", remoteAddress,
+                        namespaceName);
+            }
+            commandSender.sendErrorResponse(requestId, ServerError.TooManyRequests,
+                    "Failed due to too many pending lookup requests");
+        }
+    }
+
+    protected void handleCommandUnwatchTopicList(CommandUnwatchTopicList commandUnwatchTopicList) {
+        topicListService.handleUnwatchTopicList(commandUnwatchTopicList);

Review Comment:
   Should we check authorization here or it will be not necessary?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org