Skip to content

Commit

Permalink
[improve] PIP-241: add TopicEventListener / topic events for the Brok…
Browse files Browse the repository at this point in the history
…erService (#5)

Co-authored-by: liudezhi <[email protected]>
  • Loading branch information
278903642 and liudezhi2098 authored Dec 25, 2023
1 parent 0e6e72e commit 038c574
Show file tree
Hide file tree
Showing 6 changed files with 571 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,8 @@
import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException;
import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
import org.apache.pulsar.broker.service.TopicEventsListener.EventStage;
import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
Expand Down Expand Up @@ -287,6 +289,8 @@ public class BrokerService implements Closeable {
private Set<BrokerEntryMetadataInterceptor> brokerEntryMetadataInterceptors;
private Set<ManagedLedgerPayloadProcessor> brokerEntryPayloadProcessors;

private final TopicEventsDispatcher topicEventsDispatcher = new TopicEventsDispatcher();

public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception {
this.pulsar = pulsar;
this.preciseTopicPublishRateLimitingEnable =
Expand Down Expand Up @@ -401,6 +405,16 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws
this.bundlesQuotas = new BundlesQuotas(pulsar.getLocalMetadataStore());
}

public void addTopicEventListener(TopicEventsListener... listeners) {
topicEventsDispatcher.addTopicEventListener(listeners);
getTopics().keys().forEach(topic ->
TopicEventsDispatcher.notify(listeners, topic, TopicEvent.LOAD, EventStage.SUCCESS, null));
}

public void removeTopicEventListener(TopicEventsListener... listeners) {
topicEventsDispatcher.removeTopicEventListener(listeners);
}

// This call is used for starting additional protocol handlers
public void startProtocolHandlers(
Map<String, Map<InetSocketAddress, ChannelInitializer<SocketChannel>>> protocolHandlers) {
Expand Down Expand Up @@ -1010,17 +1024,37 @@ public CompletableFuture<Optional<Topic>> getTopic(final String topic, boolean c
});
} else {
return topics.computeIfAbsent(topic, (name) -> {
topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.BEFORE);
if (topicName.isPartitioned()) {
final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName());
return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> {
if (topicName.getPartitionIndex() < metadata.partitions) {
return createNonPersistentTopic(name);
topicEventsDispatcher
.notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE);

CompletableFuture<Optional<Topic>> res = createNonPersistentTopic(name);

CompletableFuture<Optional<Topic>> eventFuture = topicEventsDispatcher
.notifyOnCompletion(res, topicName.toString(), TopicEvent.CREATE);
topicEventsDispatcher
.notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD);
return res;
}
topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE);
return CompletableFuture.completedFuture(Optional.empty());
});
} else if (createIfMissing) {
return createNonPersistentTopic(name);
topicEventsDispatcher.notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE);

CompletableFuture<Optional<Topic>> res = createNonPersistentTopic(name);

CompletableFuture<Optional<Topic>> eventFuture = topicEventsDispatcher
.notifyOnCompletion(res, topicName.toString(), TopicEvent.CREATE);
topicEventsDispatcher
.notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD);
return res;
} else {
topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE);
return CompletableFuture.completedFuture(Optional.empty());
}
});
Expand Down Expand Up @@ -1066,6 +1100,13 @@ public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete) {
}

public CompletableFuture<Void> deleteTopic(String topic, boolean forceDelete, boolean deleteSchema) {
topicEventsDispatcher.notify(topic, TopicEvent.DELETE, EventStage.BEFORE);
CompletableFuture<Void> result = deleteTopicInternal(topic, forceDelete, deleteSchema);
topicEventsDispatcher.notifyOnCompletion(result, topic, TopicEvent.DELETE);
return result;
}

public CompletableFuture<Void> deleteTopicInternal(String topic, boolean forceDelete, boolean deleteSchema) {
Optional<Topic> optTopic = getTopicReference(topic);
if (optTopic.isPresent()) {
Topic t = optTopic.get();
Expand Down Expand Up @@ -1526,6 +1567,24 @@ private void createPersistentTopic(final String topic, boolean createIfMissing,
managedLedgerConfig.setCreateIfMissing(createIfMissing);
managedLedgerConfig.setProperties(properties);

topicEventsDispatcher.notify(topic, TopicEvent.LOAD, EventStage.BEFORE);
// load can fail with topicFuture completed non-exceptionally
// work around this
final CompletableFuture<Void> loadFuture = new CompletableFuture<>();
topicFuture.whenComplete((res, ex) -> {
if (ex == null) {
loadFuture.complete(null);
} else {
loadFuture.completeExceptionally(ex);
}
});

if (createIfMissing) {
topicEventsDispatcher.notify(topic, TopicEvent.CREATE, EventStage.BEFORE);
topicEventsDispatcher.notifyOnCompletion(topicFuture, topic, TopicEvent.CREATE);
}
topicEventsDispatcher.notifyOnCompletion(loadFuture, topic, TopicEvent.LOAD);

// Once we have the configuration, we can proceed with the async open operation
managedLedgerFactory.asyncOpen(topicName.getPersistenceNamingEncoding(), managedLedgerConfig,
new OpenLedgerCallback() {
Expand Down Expand Up @@ -1594,6 +1653,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
public void openLedgerFailed(ManagedLedgerException exception, Object ctx) {
if (!createIfMissing && exception instanceof ManagedLedgerNotFoundException) {
// We were just trying to load a topic and the topic doesn't exist
loadFuture.completeExceptionally(exception);
topicFuture.complete(Optional.empty());
} else {
log.warn("Failed to create topic {}", topic, exception);
Expand Down Expand Up @@ -2085,6 +2145,8 @@ private void removeTopicFromCache(String topic, NamespaceBundle namespaceBundle,
String bundleName = namespaceBundle.toString();
String namespaceName = TopicName.get(topic).getNamespaceObject().toString();

topicEventsDispatcher.notify(topic, TopicEvent.UNLOAD, EventStage.BEFORE);

synchronized (multiLayerTopicsMap) {
ConcurrentOpenHashMap<String, ConcurrentOpenHashMap<String, Topic>> namespaceMap = multiLayerTopicsMap
.get(namespaceName);
Expand Down Expand Up @@ -2119,6 +2181,7 @@ private void removeTopicFromCache(String topic, NamespaceBundle namespaceBundle,
if (compactor != null) {
compactor.getStats().removeTopic(topic);
}
topicEventsDispatcher.notify(topic, TopicEvent.UNLOAD, EventStage.SUCCESS);
}

public int getNumberOfNamespaceBundles() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/**
* 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.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import lombok.extern.slf4j.Slf4j;

/**
* Utility class to dispatch topic events.
*/
@Slf4j
public class TopicEventsDispatcher {
private final List<TopicEventsListener> topicEventListeners = new CopyOnWriteArrayList<>();

/**
* Adds listeners, ignores null listeners.
* @param listeners
*/
public void addTopicEventListener(TopicEventsListener... listeners) {
Objects.requireNonNull(listeners);
Arrays.stream(listeners)
.filter(x -> x != null)
.forEach(topicEventListeners::add);
}

/**
* Removes listeners.
* @param listeners
*/
public void removeTopicEventListener(TopicEventsListener... listeners) {
Objects.requireNonNull(listeners);
Arrays.stream(listeners)
.filter(x -> x != null)
.forEach(topicEventListeners::remove);
}

/**
* Dispatches notification to all currently added listeners.
* @param topic
* @param event
* @param stage
*/
public void notify(String topic,
TopicEventsListener.TopicEvent event,
TopicEventsListener.EventStage stage) {
notify(topic, event, stage, null);
}

/**
* Dispatches notification to all currently added listeners.
* @param topic
* @param event
* @param stage
* @param t
*/
public void notify(String topic,
TopicEventsListener.TopicEvent event,
TopicEventsListener.EventStage stage,
Throwable t) {
topicEventListeners
.forEach(listener -> notify(listener, topic, event, stage, t));
}

/**
* Dispatches SUCCESS/FAILURE notification to all currently added listeners on completion of the future.
* @param future
* @param topic
* @param event
* @param <T>
* @return future of a new completion stage
*/
public <T> CompletableFuture<T> notifyOnCompletion(CompletableFuture<T> future,
String topic,
TopicEventsListener.TopicEvent event) {
return future.whenComplete((r, ex) -> notify(topic,
event,
ex == null ? TopicEventsListener.EventStage.SUCCESS : TopicEventsListener.EventStage.FAILURE,
ex));
}

/**
* Dispatches notification to specified listeners.
* @param listeners
* @param topic
* @param event
* @param stage
* @param t
*/
public static void notify(TopicEventsListener[] listeners,
String topic,
TopicEventsListener.TopicEvent event,
TopicEventsListener.EventStage stage,
Throwable t) {
Objects.requireNonNull(listeners);
for (TopicEventsListener listener: listeners) {
notify(listener, topic, event, stage, t);
}
}

private static void notify(TopicEventsListener listener,
String topic,
TopicEventsListener.TopicEvent event,
TopicEventsListener.EventStage stage,
Throwable t) {
if (listener == null) {
return;
}

try {
listener.handleEvent(topic, event, stage, t);
} catch (Throwable ex) {
log.error("TopicEventsListener {} exception while handling {}_{} for topic {}",
listener, event, stage, topic, ex);
}
}

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

import org.apache.pulsar.common.classification.InterfaceAudience;
import org.apache.pulsar.common.classification.InterfaceStability;

/**
* Listener for the Topic events.
*/
@InterfaceStability.Evolving
@InterfaceAudience.LimitedPrivate
public interface TopicEventsListener {

/**
* Types of events currently supported.
* create/load/unload/delete
*/
enum TopicEvent {
// create events included into load events
CREATE,
LOAD,
UNLOAD,
DELETE,
}

/**
* Stages of events currently supported.
* before starting the event/successful completion/failed completion
*/
enum EventStage {
BEFORE,
SUCCESS,
FAILURE
}

/**
* Handle topic event.
* Choice of the thread / maintenance of the thread pool is up to the event handlers.
* @param topicName - name of the topic
* @param event - TopicEvent
* @param stage - EventStage
* @param t - exception in case of FAILURE, if present/known
*/
void handleEvent(String topicName, TopicEvent event, EventStage stage, Throwable t);
}
Loading

0 comments on commit 038c574

Please sign in to comment.