Skip to content

Commit

Permalink
[feat][broker] PIP-145: Notifications for faster topic discovery (apa…
Browse files Browse the repository at this point in the history
…che#16062)

* PIP-145: Notifications for faster topic discovery

This commit introduces topic list watchers. By using these objects
clients can observe 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.

* Fix checkstyle violation

* Fix cpp client compile error

* Remove unused code, remove failed watchers

* Rename command Unwatch, extract fields from command to avoid concurrent modification

* Fix cpp client compile error

* Fix cpp client compile error

Co-authored-by: Matteo Merli <[email protected]>
  • Loading branch information
Andras Beni and merlimat authored Jul 1, 2022
1 parent a91aa2b commit d9f5640
Show file tree
Hide file tree
Showing 30 changed files with 1,555 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,22 +19,35 @@
package org.apache.pulsar.broker.resources;

import static org.apache.pulsar.common.util.Codec.decode;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.BiConsumer;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
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.metadata.api.MetadataStore;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;

public class TopicResources {
private static final String MANAGED_LEDGER_PATH = "/managed-ledgers";

private final MetadataStore store;

private final Map<BiConsumer<String, NotificationType>, Pattern> topicListeners;

public TopicResources(MetadataStore store) {
this.store = store;
topicListeners = new ConcurrentHashMap<>();
store.registerListener(this::handleNotification);
}

public CompletableFuture<List<String>> listPersistentTopicsAsync(NamespaceName ns) {
Expand Down Expand Up @@ -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) {
return Pattern.compile(
MANAGED_LEDGER_PATH + "/(" + namespaceName + ")/(" + TopicDomain.persistent + ")/(" + "[^/]+)");
}

public void registerPersistentTopicListener(
NamespaceName namespaceName, BiConsumer<String, NotificationType> listener) {
topicListeners.put(listener, namespaceNameToTopicNamePattern(namespaceName));
}

public void deregisterPersistentTopicListener(BiConsumer<String, NotificationType> listener) {
topicListeners.remove(listener);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.resources;

import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.metadata.api.MetadataStore;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import java.util.function.BiConsumer;

public class TopicResourcesTest {

private MetadataStore metadataStore;
private TopicResources topicResources;

@BeforeMethod
public void setup() {
metadataStore = mock(MetadataStore.class);
topicResources = new TopicResources(metadataStore);
}

@Test
public void testConstructorRegistersAsListener() {
verify(metadataStore).registerListener(any());
}

@Test
public void testListenerInvokedWhenTopicCreated() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic"));
verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Created);
}

@Test
public void testListenerInvokedWhenTopicV1Created() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/cluster/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/cluster/namespace/persistent/topic"));
verify(listener).accept("persistent://tenant/cluster/namespace/topic", NotificationType.Created);
}

@Test
public void testListenerInvokedWhenTopicDeleted() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Deleted, "/managed-ledgers/tenant/namespace/persistent/topic"));
verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Deleted);
}

@Test
public void testListenerNotInvokedWhenSubscriptionCreated() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic/subscription"));
verifyNoInteractions(listener);
}

@Test
public void testListenerNotInvokedWhenTopicCreatedInOtherNamespace() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace2/persistent/topic"));
verifyNoInteractions(listener);
}

@Test
public void testListenerNotInvokedWhenTopicModified() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Modified, "/managed-ledgers/tenant/namespace/persistent/topic"));
verifyNoInteractions(listener);
}

@Test
public void testListenerNotInvokedAfterDeregistered() {
BiConsumer<String, NotificationType> listener = mock(BiConsumer.class);
topicResources.registerPersistentTopicListener(NamespaceName.get("tenant/namespace"), listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic"));
verify(listener).accept("persistent://tenant/namespace/topic", NotificationType.Created);
topicResources.deregisterPersistentTopicListener(listener);
topicResources.handleNotification(new Notification(NotificationType.Created, "/managed-ledgers/tenant/namespace/persistent/topic2"));
verifyNoMoreInteractions(listener);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ void sendGetTopicsOfNamespaceResponse(List<String> topics, String topicsHash, bo

void sendGetOrCreateSchemaErrorResponse(long requestId, ServerError error, String errorMessage);

void sendConnectedResponse(int clientProtocolVersion, int maxMessageSize);
void sendConnectedResponse(int clientProtocolVersion, int maxMessageSize, boolean supportsTopicWatchers);

void sendLookupResponse(String brokerServiceUrl, String brokerServiceUrlTls, boolean authoritative,
CommandLookupTopicResponse.LookupType response, long requestId,
Expand Down Expand Up @@ -92,4 +92,9 @@ Future<Void> sendMessagesToConsumer(long consumerId, String topicName, Subscript
void sendEndTxnResponse(long requestId, TxnID txnID, int txnAction);

void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError error, String message);

void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List<String> topics);

void sendWatchTopicListUpdate(long watcherId,
List<String> newTopics, List<String> deletedTopics, String topicsHash);
}
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,9 @@ public void sendGetOrCreateSchemaErrorResponse(long requestId, ServerError error
}

@Override
public void sendConnectedResponse(int clientProtocolVersion, int maxMessageSize) {
BaseCommand command = Commands.newConnectedCommand(clientProtocolVersion, maxMessageSize);
public void sendConnectedResponse(int clientProtocolVersion, int maxMessageSize, boolean supportsTopicWatchers) {
BaseCommand command = Commands.newConnectedCommand(
clientProtocolVersion, maxMessageSize, supportsTopicWatchers);
safeIntercept(command, cnx);
ByteBuf outBuf = Commands.serializeWithSize(command);
cnx.ctx().writeAndFlush(outBuf, cnx.ctx().voidPromise());
Expand Down Expand Up @@ -346,6 +347,25 @@ public void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError err
}
}

@Override
public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List<String> topics) {
BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics);
interceptAndWriteCommand(command);
}

@Override
public void sendWatchTopicListUpdate(long watcherId,
List<String> newTopics, List<String> deletedTopics, String topicsHash) {
BaseCommand command = Commands.newWatchTopicUpdate(watcherId, newTopics, deletedTopics, topicsHash);
interceptAndWriteCommand(command);
}

private void interceptAndWriteCommand(BaseCommand command) {
safeIntercept(command, cnx);
ByteBuf outBuf = Commands.serializeWithSize(command);
cnx.ctx().writeAndFlush(outBuf);
}

private void safeIntercept(BaseCommand command, ServerCnx cnx) {
try {
this.interceptor.onPulsarCommand(command, cnx);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.naming.AuthenticationException;
import javax.net.ssl.SSLSession;
Expand Down Expand Up @@ -119,6 +120,8 @@
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.CommandTcClientConnectRequest;
import org.apache.pulsar.common.api.proto.CommandUnsubscribe;
import org.apache.pulsar.common.api.proto.CommandWatchTopicList;
import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose;
import org.apache.pulsar.common.api.proto.FeatureFlags;
import org.apache.pulsar.common.api.proto.KeySharedMeta;
import org.apache.pulsar.common.api.proto.KeySharedMode;
Expand Down Expand Up @@ -164,6 +167,7 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
private final ConcurrentLongHashMap<CompletableFuture<Consumer>> consumers;
private final boolean enableSubscriptionPatternEvaluation;
private final int maxSubscriptionPatternLength;
private final TopicListService topicListService;
private State state;
private volatile boolean isActive = true;
private String authRole = null;
Expand Down Expand Up @@ -272,6 +276,8 @@ public ServerCnx(PulsarService pulsar, String listenerName) {
this.connectionController = new ConnectionController.DefaultConnectionController(conf);
this.enableSubscriptionPatternEvaluation = conf.isEnableBrokerSideSubscriptionPatternEvaluation();
this.maxSubscriptionPatternLength = conf.getSubscriptionPatternMaxLength();
this.topicListService = new TopicListService(pulsar, this,
enableSubscriptionPatternEvaluation, maxSubscriptionPatternLength);
}

@Override
Expand Down Expand Up @@ -334,6 +340,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
}
}
});
this.topicListService.inactivate();
this.service.getPulsarStats().recordConnectionClose();
}

Expand Down Expand Up @@ -625,8 +632,8 @@ ByteBuf createConsumerStatsResponse(Consumer consumer, long requestId) {
}

// complete the connect and sent newConnected command
private void completeConnect(int clientProtoVersion, String clientVersion) {
ctx.writeAndFlush(Commands.newConnected(clientProtoVersion, maxMessageSize));
private void completeConnect(int clientProtoVersion, String clientVersion, boolean supportsTopicWatchers) {
ctx.writeAndFlush(Commands.newConnected(clientProtoVersion, maxMessageSize, supportsTopicWatchers));
state = State.Connected;
service.getPulsarStats().recordConnectionCreateSuccess();
if (log.isDebugEnabled()) {
Expand Down Expand Up @@ -685,7 +692,7 @@ private State doAuthentication(AuthData clientData,

if (state != State.Connected) {
// First time authentication is done
completeConnect(clientProtocolVersion, clientVersion);
completeConnect(clientProtocolVersion, clientVersion, enableSubscriptionPatternEvaluation);
} else {
// If the connection was already ready, it means we're doing a refresh
if (!StringUtils.isEmpty(authRole)) {
Expand Down Expand Up @@ -792,7 +799,7 @@ protected void handleConnect(CommandConnect connect) {
}

if (!service.isAuthenticationEnabled()) {
completeConnect(clientProtocolVersion, clientVersion);
completeConnect(clientProtocolVersion, clientVersion, enableSubscriptionPatternEvaluation);
return;
}

Expand Down Expand Up @@ -820,7 +827,7 @@ protected void handleConnect(CommandConnect connect) {
authRole = getBrokerService().getAuthenticationService().getAnonymousUserRole()
.orElseThrow(() ->
new AuthenticationException("No anonymous role, and no authentication provider configured"));
completeConnect(clientProtocolVersion, clientVersion);
completeConnect(clientProtocolVersion, clientVersion, enableSubscriptionPatternEvaluation);
return;
}

Expand Down Expand Up @@ -2527,6 +2534,59 @@ 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());

Pattern topicsPattern = Pattern.compile(commandWatchTopicList.hasTopicsPattern()
? commandWatchTopicList.getTopicsPattern() : TopicList.ALL_TOPICS_PATTERN);
String topicsHash = commandWatchTopicList.hasTopicsHash()
? commandWatchTopicList.getTopicsHash() : null;

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(namespaceName, watcherId, requestId, topicsPattern,
topicsHash, 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 handleCommandWatchTopicListClose(CommandWatchTopicListClose commandWatchTopicListClose) {
topicListService.handleWatchTopicListClose(commandWatchTopicListClose);
}

@Override
protected boolean isHandshakeCompleted() {
return state == State.Connected;
Expand Down
Loading

0 comments on commit d9f5640

Please sign in to comment.