diff --git a/conf/broker.conf b/conf/broker.conf index 74130d709cdd2..125b2aa8c1b39 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1563,6 +1563,15 @@ loadBalancerNamespaceBundleSplitConditionHitCountThreshold=3 # (only used in load balancer extension logics) loadBalancerServiceUnitStateTombstoneDelayTimeInSeconds=3600 +# Name of ServiceUnitStateTableView implementation class to use +loadManagerServiceUnitStateTableViewClassName=org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl + +# Specify ServiceUnitTableViewSyncer to sync service unit(bundle) states between metadata store and +# system topic table views during migration from one to the other. One could enable this +# syncer before migration and disable it after the migration finishes. +# It accepts `MetadataStoreToSystemTopicSyncer` or `SystemTopicToMetadataStoreSyncer` to +# enable it. It accepts `None` to disable it." +loadBalancerServiceUnitTableViewSyncer=None ### --- Replication --- ### diff --git a/pip/pip-378.md b/pip/pip-378.md index 352c7fa560d1c..e44ce7339cf53 100644 --- a/pip/pip-378.md +++ b/pip/pip-378.md @@ -30,7 +30,7 @@ Add `ServiceUnitStateTableView` abstraction and make it pluggable, so users can - Introduce `MetadataStoreTableView` interface to support `ServiceUnitStateMetadataStoreTableViewImpl` implementation. - `MetadataStoreTableViewImpl` will use shadow hashmap to maintain the metadata tableview. It will initially fill the local tableview by scanning all existing items in the metadata store path. Also, new items will be updated to the tableview via metadata watch notifications. - Add `BiConsumer>> asyncReloadConsumer` in MetadataCacheConfig to listen the automatic cache async reload. This can be useful to re-sync the the shadow hashmap in MetadataStoreTableViewImpl in case it is out-dated in the worst case(e.g. network or metadata issues). -- Introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views to migrate to one from the other. This syncer can be enabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. +- Introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views to migrate to one from the other. This syncer can be enabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncer`. ## Detailed Design @@ -243,56 +243,15 @@ public class MetadataCacheConfig { */ @Slf4j public class ServiceUnitStateTableViewSyncer implements Cloneable { - private static final int SYNC_TIMEOUT_IN_SECS = 30; - private volatile ServiceUnitStateTableView systemTopicTableView; - private volatile ServiceUnitStateTableView metadataStoreTableView; - + ... public void start(PulsarService pulsar) throws IOException { - if (!pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { - return; - } - try { - if (systemTopicTableView == null) { - systemTopicTableView = new ServiceUnitStateTableViewImpl(); - systemTopicTableView.start( - pulsar, - this::syncToMetadataStore, - this::syncToMetadataStore); - log.info("Successfully started ServiceUnitStateTableViewSyncer::systemTopicTableView"); - } - - if (metadataStoreTableView == null) { - metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); - metadataStoreTableView.start( - pulsar, - this::syncToSystemTopic, - this::syncToSystemTopic); - log.info("Successfully started ServiceUnitStateTableViewSyncer::metadataStoreTableView"); - } - - } catch (Throwable e) { - log.error("Failed to start ServiceUnitStateTableViewSyncer", e); - throw e; - } + ... // sync SystemTopicTableView and MetadataStoreTableView } - private void syncToSystemTopic(String key, ServiceUnitStateData data) { - try { - systemTopicTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); - } catch (Throwable e) { - log.error("SystemTopicTableView failed to sync key:{}, data:{}", key, data, e); - throw new IllegalStateException(e); - } - } - private void syncToMetadataStore(String key, ServiceUnitStateData data) { - try { - metadataStoreTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); - } catch (Throwable e) { - log.error("metadataStoreTableView failed to sync key:{}, data:{}", key, data, e); - throw new IllegalStateException(e); - } + public void close() throws IOException { + ... // stop syncer } ... } @@ -302,14 +261,14 @@ public class ServiceUnitStateTableViewSyncer implements Cloneable { ### Configuration -- Add a `loadManagerServiceUnitStateTableViewClassName` configuration to specify `ServiceUnitStateTableView` implementation class name. -- Add a `loadBalancerServiceUnitTableViewSyncerEnabled` configuration to to enable ServiceUnitTableViewSyncer to sync metadata store and system topic ServiceUnitStateTableView during migration. +- Add a `loadManagerServiceUnitStateTableViewClassName` static configuration to specify `ServiceUnitStateTableView` implementation class name. +- Add a `loadBalancerServiceUnitTableViewSyncer` dynamic configuration to enable ServiceUnitTableViewSyncer to sync metadata store and system topic ServiceUnitStateTableView during migration. ## Backward & Forward Compatibility It will ba Backward & Forward compatible as `loadManagerServiceUnitStateTableViewClassName` will be `ServiceUnitStateTableViewImpl`(system topic implementation) by default. -We will introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views when migrating to ServiceUnitStateMetadataStoreTableViewImpl from ServiceUnitStateTableViewImpl and vice versa. This syncer can be enabled/disabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. The admin could enable this syncer before migration and disable it after it is finished. +We will introduce `ServiceUnitStateTableViewSyncer` dynamic config to sync system topic and metadata store table views when migrating to ServiceUnitStateMetadataStoreTableViewImpl from ServiceUnitStateTableViewImpl and vice versa. The admin could enable this syncer before migration and disable it after it is finished. ## Alternatives diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index cdd27412e3052..486587ec174a0 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2912,6 +2912,25 @@ public double getLoadBalancerBandwidthOutResourceWeight() { ) private boolean loadBalancerMultiPhaseBundleUnload = true; + @FieldContext( + dynamic = false, + category = CATEGORY_LOAD_BALANCER, + doc = "Name of ServiceUnitStateTableView implementation class to use" + ) + private String loadManagerServiceUnitStateTableViewClassName = + "org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl"; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "Specify ServiceUnitTableViewSyncer to sync service unit(bundle) states between metadata store and " + + "system topic table views during migration from one to the other. One could enable this" + + " syncer before migration and disable it after the migration finishes. " + + "It accepts `MetadataStoreToSystemTopicSyncer` or `SystemTopicToMetadataStoreSyncer` to " + + "enable it. It accepts `None` to disable it." + ) + private ServiceUnitTableViewSyncerType loadBalancerServiceUnitTableViewSyncer = ServiceUnitTableViewSyncerType.None; + /**** --- Replication. --- ****/ @FieldContext( category = CATEGORY_REPLICATION, @@ -3810,4 +3829,14 @@ public Map lookupProperties() { }); return map; } + + public boolean isLoadBalancerServiceUnitTableViewSyncerEnabled() { + return loadBalancerServiceUnitTableViewSyncer != ServiceUnitTableViewSyncerType.None; + } + + public enum ServiceUnitTableViewSyncerType { + None, + MetadataStoreToSystemTopicSyncer, + SystemTopicToMetadataStoreSyncer; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index f22bcc836f6e9..98ef6bf36edac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -21,10 +21,9 @@ import static java.lang.String.format; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Follower; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Leader; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.TOPIC; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Label.Success; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; -import static org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared.getNamespaceBundle; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; @@ -41,20 +40,17 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import java.util.stream.Collectors; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewSyncer; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -172,6 +168,9 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private TopBundleLoadDataReporter topBundleLoadDataReporter; + @Getter + protected ServiceUnitStateTableViewSyncer serviceUnitStateTableViewSyncer; + private volatile ScheduledFuture brokerLoadDataReportTask; private volatile ScheduledFuture topBundlesLoadDataReportTask; @@ -209,46 +208,18 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS /** * Get all the bundles that are owned by this broker. */ + @Deprecated public CompletableFuture> getOwnedServiceUnitsAsync() { + return CompletableFuture.completedFuture(getOwnedServiceUnits()); + } + + public Set getOwnedServiceUnits() { if (!started) { log.warn("Failed to get owned service units, load manager is not started."); - return CompletableFuture.completedFuture(Collections.emptySet()); + return Collections.emptySet(); } - String brokerId = brokerRegistry.getBrokerId(); - Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); - Set ownedServiceUnits = entrySet.stream() - .filter(entry -> { - var stateData = entry.getValue(); - return stateData.state() == ServiceUnitState.Owned - && StringUtils.isNotBlank(stateData.dstBroker()) - && stateData.dstBroker().equals(brokerId); - }).map(entry -> { - var bundle = entry.getKey(); - return getNamespaceBundle(pulsar, bundle); - }).collect(Collectors.toSet()); - // Add heartbeat and SLA monitor namespace bundle. - NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); - NamespaceName heartbeatNamespaceV2 = NamespaceService - .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); - NamespaceName slaMonitorNamespace = NamespaceService - .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); - return pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(heartbeatNamespace) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get heartbeat namespace bundle.", e); - return null; - }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(heartbeatNamespaceV2)) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get heartbeat namespace V2 bundle.", e); - return null; - }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(slaMonitorNamespace)) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get SLA Monitor namespace bundle.", e); - return null; - }).thenApply(__ -> ownedServiceUnits); + return serviceUnitStateChannel.getOwnedServiceUnits(); } @Override @@ -317,14 +288,14 @@ private static void createSystemTopics(PulsarService pulsar) throws PulsarServer createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } - private static boolean configureSystemTopics(PulsarService pulsar) { + public static boolean configureSystemTopics(PulsarService pulsar, long target) { try { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) && pulsar.getConfiguration().isTopicLevelPoliciesEnabled()) { Long threshold = pulsar.getAdminClient().topicPolicies().getCompactionThreshold(TOPIC); - if (threshold == null || COMPACTION_THRESHOLD != threshold.longValue()) { - pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, COMPACTION_THRESHOLD); - log.info("Set compaction threshold: {} bytes for system topic {}.", COMPACTION_THRESHOLD, TOPIC); + if (threshold == null || target != threshold.longValue()) { + pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, target); + log.info("Set compaction threshold: {} bytes for system topic {}.", target, TOPIC); } } else { log.warn("System topic or topic level policies is disabled. " @@ -432,6 +403,7 @@ public void start() throws PulsarServerException { serviceUnitStateChannel, unloadCounter, unloadMetrics); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); + this.serviceUnitStateTableViewSyncer = new ServiceUnitStateTableViewSyncer(); pulsar.runWhenReadyForIncomingRequests(() -> { try { @@ -799,10 +771,11 @@ public void close() throws PulsarServerException { monitorTask.cancel(true); } - this.brokerLoadDataStore.close(); - this.topBundlesLoadDataStore.close(); + this.brokerLoadDataStore.shutdown(); + this.topBundlesLoadDataStore.shutdown(); this.unloadScheduler.close(); this.splitScheduler.close(); + this.serviceUnitStateTableViewSyncer.close(); } catch (IOException ex) { throw new PulsarServerException(ex); } finally { @@ -857,6 +830,9 @@ synchronized void playLeader() { topBundlesLoadDataStore.init(); unloadScheduler.start(); serviceUnitStateChannel.scheduleOwnershipMonitor(); + if (pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + serviceUnitStateTableViewSyncer.start(pulsar); + } break; } catch (Throwable e) { log.warn("The broker:{} failed to set the role. Retrying {} th ...", @@ -906,6 +882,7 @@ synchronized void playFollower() { brokerLoadDataStore.init(); topBundlesLoadDataStore.close(); topBundlesLoadDataStore.startProducer(); + serviceUnitStateTableViewSyncer.close(); break; } catch (Throwable e) { log.warn("The broker:{} failed to set the role. Retrying {} th ...", @@ -977,19 +954,27 @@ protected void monitor() { // System topic config might fail due to the race condition // with topic policy init(Topic policies cache have not init). if (!configuredSystemTopics) { - configuredSystemTopics = configureSystemTopics(pulsar); + configuredSystemTopics = configureSystemTopics(pulsar, COMPACTION_THRESHOLD); } if (role != Leader) { log.warn("Current role:{} does not match with the channel ownership:{}. " + "Playing the leader role.", role, isChannelOwner); playLeader(); } + + if (pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + serviceUnitStateTableViewSyncer.start(pulsar); + } else { + serviceUnitStateTableViewSyncer.close(); + } + } else { if (role != Follower) { log.warn("Current role:{} does not match with the channel ownership:{}. " + "Playing the follower role.", role, isChannelOwner); playFollower(); } + serviceUnitStateTableViewSyncer.close(); } } catch (Throwable e) { log.error("Failed to get the channel ownership.", e); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java index 42ef55593ae1a..b823a8277d376 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java @@ -42,7 +42,13 @@ public enum ServiceUnitState { Deleted; // deleted in the system (semi-terminal state) - private static final Map> validTransitions = Map.of( + + public enum StorageType { + SystemTopic, + MetadataStore; + } + + private static final Map> validTransitionsOverSystemTopic = Map.of( // (Init -> all states) transitions are required // when the topic is compacted in the middle of assign, transfer or split. Init, Set.of(Free, Owned, Assigning, Releasing, Splitting, Deleted), @@ -54,12 +60,24 @@ public enum ServiceUnitState { Deleted, Set.of(Init) ); + private static final Map> validTransitionsOverMetadataStore = Map.of( + Init, Set.of(Assigning), + Free, Set.of(Assigning), + Owned, Set.of(Splitting, Releasing), + Assigning, Set.of(Owned), + Releasing, Set.of(Assigning, Free), + Splitting, Set.of(Deleted), + Deleted, Set.of(Init) + ); + private static final Set inFlightStates = Set.of( Assigning, Releasing, Splitting ); - public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to) { - Set transitions = validTransitions.get(from); + public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to, StorageType storageType) { + Set transitions = + (storageType == StorageType.SystemTopic) ? validTransitionsOverSystemTopic.get(from) + : validTransitionsOverMetadataStore.get(from); return transitions.contains(to); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java index 6319fc332a678..ac9897a20e75c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java @@ -24,13 +24,14 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.manager.StateChangeListener; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; +import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.metadata.api.NotificationType; -import org.apache.pulsar.metadata.api.extended.SessionEvent; /** * Defines the ServiceUnitStateChannel interface. @@ -56,92 +57,39 @@ public interface ServiceUnitStateChannel extends Closeable { void close() throws PulsarServerException; /** - * Asynchronously gets the current owner broker of the system topic in this channel. - * @return the service url without the protocol prefix, 'http://'. e.g. broker-xyz:abcd - * - * ServiceUnitStateChannel elects the separate leader as the owner broker of the system topic in this channel. + * Asynchronously gets the current owner broker of this channel. + * @return a future of owner brokerId to track the completion of the operation */ CompletableFuture> getChannelOwnerAsync(); /** - * Asynchronously checks if the current broker is the owner broker of the system topic in this channel. - * @return True if the current broker is the owner. Otherwise, false. + * Asynchronously checks if the current broker is the owner broker of this channel. + * @return a future of check result to track the completion of the operation */ CompletableFuture isChannelOwnerAsync(); /** - * Checks if the current broker is the owner broker of the system topic in this channel. + * Checks if the current broker is the owner broker of this channel. * @return True if the current broker is the owner. Otherwise, false. + * @throws ExecutionException + * @throws InterruptedException + * @throws TimeoutException */ - boolean isChannelOwner(); - - /** - * Handles the metadata session events to track - * if the connection between the broker and metadata store is stable or not. - * This will be registered as a metadata SessionEvent listener. - * - * The stability of the metadata connection is important - * to determine how to handle the broker deletion(unavailable) event notified from the metadata store. - * - * Please refer to handleBrokerRegistrationEvent(String broker, NotificationType type) for more details. - * - * @param event metadata session events - */ - void handleMetadataSessionEvent(SessionEvent event); - - /** - * Handles the broker registration event from the broker registry. - * This will be registered as a broker registry listener. - * - * Case 1: If NotificationType is Deleted, - * it will schedule a clean-up operation to release the ownerships of the deleted broker. - * - * Sub-case1: If the metadata connection has been stable for long time, - * it will immediately execute the cleanup operation to guarantee high-availability. - * - * Sub-case2: If the metadata connection has been stable only for short time, - * it will defer the clean-up operation for some time and execute it. - * This is to gracefully handle the case when metadata connection is flaky -- - * If the deleted broker comes back very soon, - * we better cancel the clean-up operation for high-availability. - * - * Sub-case3: If the metadata connection is unstable, - * it will not schedule the clean-up operation, as the broker-metadata connection is lost. - * The brokers will continue to serve existing topics connections, - * and we better not to interrupt the existing topic connections for high-availability. - * - * - * Case 2: If NotificationType is Created, - * it will cancel any scheduled clean-up operation if still not executed. - * - * @param broker notified broker - * @param type notification type - */ - void handleBrokerRegistrationEvent(String broker, NotificationType type); + boolean isChannelOwner() throws ExecutionException, InterruptedException, TimeoutException; /** * Asynchronously gets the current owner broker of the service unit. * - * * @param serviceUnit (e.g. bundle) - * @return the future object of the owner broker - * - * Case 1: If the service unit is owned, it returns the completed future object with the current owner. - * Case 2: If the service unit's assignment is ongoing, it returns the non-completed future object. - * Sub-case1: If the assigned broker is available and finally takes the ownership, - * the future object will complete and return the owner broker. - * Sub-case2: If the assigned broker does not take the ownership in time, - * the future object will time out. - * Case 3: If none of them, it returns Optional.empty(). + * @return a future of owner brokerId to track the completion of the operation */ CompletableFuture> getOwnerAsync(String serviceUnit); /** - * Gets the assigned broker of the service unit. - * + * Asynchronously gets the assigned broker of the service unit. * * @param serviceUnit (e.g. bundle)) - * @return the future object of the assigned broker + * @return assigned brokerId */ Optional getAssigned(String serviceUnit); @@ -149,47 +97,39 @@ public interface ServiceUnitStateChannel extends Closeable { /** * Checks if the target broker is the owner of the service unit. * - * * @param serviceUnit (e.g. bundle) - * @param targetBroker - * @return true if the target broker is the owner. false if unknown. + * @param targetBrokerId + * @return true if the target brokerId is the owner brokerId. false if unknown. */ - boolean isOwner(String serviceUnit, String targetBroker); + boolean isOwner(String serviceUnit, String targetBrokerId); /** * Checks if the current broker is the owner of the service unit. * - * * @param serviceUnit (e.g. bundle)) * @return true if the current broker is the owner. false if unknown. */ boolean isOwner(String serviceUnit); /** - * Asynchronously publishes the service unit assignment event to the system topic in this channel. - * It de-duplicates assignment events if there is any ongoing assignment event for the same service unit. + * Asynchronously publishes the service unit assignment event to this channel. * @param serviceUnit (e.g bundle) - * @param broker the assigned broker - * @return the completable future object with the owner broker - * case 1: If the assigned broker is available and takes the ownership, - * the future object will complete and return the owner broker. - * The returned owner broker could be different from the input broker (due to assignment race-condition). - * case 2: If the assigned broker does not take the ownership in time, - * the future object will time out. + * @param brokerId the assigned brokerId + * @return a future of owner brokerId to track the completion of the operation */ - CompletableFuture publishAssignEventAsync(String serviceUnit, String broker); + CompletableFuture publishAssignEventAsync(String serviceUnit, String brokerId); /** - * Asynchronously publishes the service unit unload event to the system topic in this channel. + * Asynchronously publishes the service unit unload event to this channel. * @param unload (unload specification object) - * @return the completable future object staged from the event message sendAsync. + * @return a future to track the completion of the operation */ CompletableFuture publishUnloadEventAsync(Unload unload); /** - * Asynchronously publishes the bundle split event to the system topic in this channel. + * Asynchronously publishes the bundle split event to this channel. * @param split (split specification object) - * @return the completable future object staged from the event message sendAsync. + * @return a future to track the completion of the operation */ CompletableFuture publishSplitEventAsync(Split split); @@ -200,18 +140,24 @@ public interface ServiceUnitStateChannel extends Closeable { List getMetrics(); /** - * Add a state change listener. + * Adds a state change listener. * * @param listener State change listener. */ void listen(StateChangeListener listener); /** - * Returns service unit ownership entry set. - * @return a set of service unit ownership entries + * Asynchronously returns service unit ownership entry set. + * @return a set of service unit ownership entries to track the completion of the operation */ Set> getOwnershipEntrySet(); + /** + * Asynchronously returns service units owned by this broker. + * @return a set of owned service units to track the completion of the operation + */ + Set getOwnedServiceUnits(); + /** * Schedules ownership monitor to periodically check and correct invalid ownership states. */ @@ -223,7 +169,7 @@ public interface ServiceUnitStateChannel extends Closeable { void cancelOwnershipMonitor(); /** - * Cleans the service unit ownerships from the current broker's channel. + * Cleans(gives up) any service unit ownerships from this broker. */ void cleanOwnerships(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 5893fc4924413..ddbc9eacac921 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -32,6 +32,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isInFlightState; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Closed; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Constructed; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.LeaderElectionServiceStarted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Started; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Assign; @@ -42,7 +43,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Unstable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; -import static org.apache.pulsar.common.topics.TopicCompactionStrategy.TABLE_VIEW_TAG; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionReestablished; import com.google.common.annotations.VisibleForTesting; @@ -86,39 +86,27 @@ import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.naming.NamespaceBundles; -import org.apache.pulsar.common.naming.TopicDomain; -import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.SessionEvent; @Slf4j public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { - public static final String TOPIC = TopicName.get( - TopicDomain.persistent.value(), - SYSTEM_NAMESPACE, - "loadbalancer-service-unit-state").toString(); - public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; private static final int OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS = 5000; private static final int OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS = 100; public static final long VERSION_ID_INIT = 1; // initial versionId public static final long MAX_CLEAN_UP_DELAY_TIME_IN_SECS = 3 * 60; // 3 mins private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately private static final long MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS = 10; - private static final int MAX_OUTSTANDING_PUB_MESSAGES = 500; private static final long MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS = 10 * 60 * 1000; private final PulsarService pulsar; private final ServiceConfiguration config; @@ -129,8 +117,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final StateChangeListeners stateChangeListeners; private BrokerRegistry brokerRegistry; private LeaderElectionService leaderElectionService; - private TableView tableview; - private Producer producer; + + private ServiceUnitStateTableView tableview; private ScheduledFuture monitorTask; private SessionEvent lastMetadataSessionEvent = SessionReestablished; private long lastMetadataSessionEventTimestamp = 0; @@ -166,7 +154,8 @@ public enum EventType { public static class Counters { private final AtomicLong total; private final AtomicLong failure; - public Counters(){ + + public Counters() { total = new AtomicLong(); failure = new AtomicLong(); } @@ -181,11 +170,13 @@ enum ChannelState { Closed(0), Constructed(1), LeaderElectionServiceStarted(2), - Started(3); + Started(3), + Disabled(4); ChannelState(int id) { this.id = id; } + int id; } @@ -234,6 +225,7 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.channelState = Constructed; } + @Override public void scheduleOwnershipMonitor() { if (monitorTask == null) { this.monitorTask = this.pulsar.getLoadManagerExecutor() @@ -251,6 +243,7 @@ public void scheduleOwnershipMonitor() { } } + @Override public void cancelOwnershipMonitor() { if (monitorTask != null) { monitorTask.cancel(false); @@ -262,7 +255,7 @@ public void cancelOwnershipMonitor() { @Override public void cleanOwnerships() { - doCleanup(brokerId); + doCleanup(brokerId, true); } @Override @@ -270,6 +263,22 @@ public synchronized boolean started() { return validateChannelState(Started, true); } + private ServiceUnitStateTableView createServiceUnitStateTableView() { + ServiceConfiguration conf = pulsar.getConfiguration(); + try { + ServiceUnitStateTableView tableview = + Reflections.createInstance(conf.getLoadManagerServiceUnitStateTableViewClassName(), + ServiceUnitStateTableView.class, Thread.currentThread().getContextClassLoader()); + log.info("Created service unit state tableview: {}", tableview.getClass().getCanonicalName()); + return tableview; + } catch (Throwable e) { + log.error("Error when trying to create service unit state tableview: {}.", + conf.getLoadManagerServiceUnitStateTableViewClassName(), e); + throw e; + } + } + + @Override public synchronized void start() throws PulsarServerException { if (!validateChannelState(LeaderElectionServiceStarted, false)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); @@ -289,55 +298,17 @@ public synchronized void start() throws PulsarServerException { } this.channelState = LeaderElectionServiceStarted; - if (producer != null) { - producer.close(); - if (debug) { - log.info("Closed the channel producer."); - } - } - PulsarClusterMetadataSetup.createTenantIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), config.getClusterName()); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), + pulsar.getConfiguration().getClusterName()); PulsarClusterMetadataSetup.createNamespaceIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName(), - config.getDefaultNumberOfNamespaceBundles()); - - ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, pulsar.getConfiguration().getClusterName(), + pulsar.getConfiguration().getDefaultNumberOfNamespaceBundles()); - producer = pulsar.getClient().newProducer(schema) - .enableBatching(true) - .compressionType(MSG_COMPRESSION_TYPE) - .maxPendingMessages(MAX_OUTSTANDING_PUB_MESSAGES) - .blockIfQueueFull(true) - .topic(TOPIC) - .create(); - - if (debug) { - log.info("Successfully started the channel producer."); - } + tableview = createServiceUnitStateTableView(); + tableview.start(pulsar, this::handleEvent, this::handleExisting); - if (tableview != null) { - tableview.close(); - if (debug) { - log.info("Closed the channel tableview."); - } - } - tableview = pulsar.getClient().newTableViewBuilder(schema) - .topic(TOPIC) - .loadConf(Map.of( - "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) - .create(); - tableview.listen(this::handleEvent); - tableview.forEach(this::handleExisting); - var strategy = (ServiceUnitStateCompactionStrategy) TopicCompactionStrategy.getInstance(TABLE_VIEW_TAG); - if (strategy == null) { - String err = TABLE_VIEW_TAG + "tag TopicCompactionStrategy is null."; - log.error(err); - throw new IllegalStateException(err); - } - strategy.setSkippedMsgHandler((key, value) -> handleSkippedEvent(key)); if (debug) { log.info("Successfully started the channel tableview."); } @@ -378,23 +349,15 @@ protected LeaderElectionService getLeaderElectionService() { .get().getLeaderElectionService(); } + @Override public synchronized void close() throws PulsarServerException { channelState = Closed; - boolean debug = debug(); try { leaderElectionService = null; + if (tableview != null) { tableview.close(); tableview = null; - if (debug) { - log.info("Successfully closed the channel tableview."); - } - } - - if (producer != null) { - producer.close(); - producer = null; - log.info("Successfully closed the channel producer."); } if (brokerRegistry != null) { @@ -432,6 +395,7 @@ private boolean debug() { return ExtensibleLoadManagerImpl.debug(config, log); } + @Override public CompletableFuture> getChannelOwnerAsync() { if (!validateChannelState(LeaderElectionServiceStarted, true)) { return CompletableFuture.failedFuture( @@ -442,6 +406,7 @@ public CompletableFuture> getChannelOwnerAsync() { .thenApply(leader -> leader.map(LeaderBroker::getBrokerId)); } + @Override public CompletableFuture isChannelOwnerAsync() { return getChannelOwnerAsync().thenApply(owner -> { if (owner.isPresent()) { @@ -453,19 +418,14 @@ public CompletableFuture isChannelOwnerAsync() { } }); } - - public boolean isChannelOwner() { - try { - return isChannelOwnerAsync().get( - MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS); - } catch (InterruptedException | ExecutionException | TimeoutException e) { - String msg = "Failed to get the channel owner."; - log.error(msg, e); - throw new RuntimeException(msg, e); - } + @Override + public boolean isChannelOwner() throws ExecutionException, InterruptedException, TimeoutException { + return isChannelOwnerAsync().get( + MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS); } - public boolean isOwner(String serviceUnit, String targetBroker) { + @Override + public boolean isOwner(String serviceUnit, String targetBrokerId) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } @@ -474,12 +434,13 @@ public boolean isOwner(String serviceUnit, String targetBroker) { return false; } var owner = ownerFuture.join(); - if (owner.isPresent() && StringUtils.equals(targetBroker, owner.get())) { + if (owner.isPresent() && StringUtils.equals(targetBrokerId, owner.get())) { return true; } return false; } + @Override public boolean isOwner(String serviceUnit) { return isOwner(serviceUnit, brokerId); } @@ -512,13 +473,22 @@ private CompletableFuture> getActiveOwnerAsync( }).thenApply(Optional::ofNullable); } + /** + * Case 1: If the service unit is owned, it returns the completed future object with the current owner. + * Case 2: If the service unit's assignment is ongoing, it returns the non-completed future object. + * Sub-case1: If the assigned broker is available and finally takes the ownership, + * the future object will complete and return the owner broker. + * Sub-case2: If the assigned broker does not take the ownership in time, + * the future object will time out. + * Case 3: If none of them, it returns Optional.empty(). + */ + @Override public CompletableFuture> getOwnerAsync(String serviceUnit) { if (!validateChannelState(Started, true)) { return CompletableFuture.failedFuture( new IllegalStateException("Invalid channel state:" + channelState.name())); } - - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); ServiceUnitState state = state(data); ownerLookUpCounters.get(state).getTotal().incrementAndGet(); switch (state) { @@ -549,18 +519,19 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } default -> { ownerLookUpCounters.get(state).getFailure().incrementAndGet(); - String errorMsg = String.format("Failed to process service unit state data: %s when get owner.", data); + String errorMsg = + String.format("Failed to process service unit state data: %s when get owner.", data); log.error(errorMsg); return CompletableFuture.failedFuture(new IllegalStateException(errorMsg)); } } } - private Optional getOwner(String serviceUnit) { + private Optional getOwnerNow(String serviceUnit) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); ServiceUnitState state = state(data); switch (state) { case Owned -> { @@ -578,13 +549,14 @@ private Optional getOwner(String serviceUnit) { } } + @Override public Optional getAssigned(String serviceUnit) { if (!validateChannelState(Started, true)) { return Optional.empty(); } - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); if (data == null) { return Optional.empty(); } @@ -607,22 +579,23 @@ public Optional getAssigned(String serviceUnit) { return Optional.empty(); } default -> { - log.warn("Trying to get the assigned broker from unknown state:{} serviceUnit:{}", state, serviceUnit); + log.warn("Trying to get the assigned broker from unknown state:{} serviceUnit:{}", state, + serviceUnit); return Optional.empty(); } } } - private long getNextVersionId(String serviceUnit) { - var data = tableview.get(serviceUnit); - return getNextVersionId(data); + private Long getNextVersionId(String serviceUnit) { + return getNextVersionId(tableview.get(serviceUnit)); } private long getNextVersionId(ServiceUnitStateData data) { return data == null ? VERSION_ID_INIT : data.versionId() + 1; } - public CompletableFuture publishAssignEventAsync(String serviceUnit, String broker) { + @Override + public CompletableFuture publishAssignEventAsync(String serviceUnit, String brokerId) { if (!validateChannelState(Started, true)) { return CompletableFuture.failedFuture( new IllegalStateException("Invalid channel state:" + channelState.name())); @@ -631,7 +604,8 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str eventCounters.get(eventType).getTotal().incrementAndGet(); CompletableFuture getOwnerRequest = dedupeGetOwnerRequest(serviceUnit); - pubAsync(serviceUnit, new ServiceUnitStateData(Assigning, broker, getNextVersionId(serviceUnit))) + pubAsync(serviceUnit, + new ServiceUnitStateData(Assigning, brokerId, getNextVersionId(serviceUnit))) .whenComplete((__, ex) -> { if (ex != null) { getOwnerRequests.remove(serviceUnit, getOwnerRequest); @@ -641,11 +615,12 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str eventCounters.get(eventType).getFailure().incrementAndGet(); } }); + return getOwnerRequest; } private CompletableFuture publishOverrideEventAsync(String serviceUnit, - ServiceUnitStateData override) { + ServiceUnitStateData override) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } @@ -697,12 +672,17 @@ public CompletableFuture publishSplitEventAsync(Split split) { } private void handleEvent(String serviceUnit, ServiceUnitStateData data) { + long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet(); if (debug()) { log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}", brokerId, serviceUnit, data, totalHandledRequests); } + if (channelState == Disabled) { + return; + } + ServiceUnitState state = state(data); try { switch (state) { @@ -715,7 +695,7 @@ private void handleEvent(String serviceUnit, ServiceUnitStateData data) { case Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } - } catch (Throwable e){ + } catch (Throwable e) { log.error("Failed to handle the event. serviceUnit:{}, data:{}, handlerFailureCount:{}", serviceUnit, data, getHandlerFailureCounter(data).incrementAndGet(), e); throw e; @@ -914,26 +894,24 @@ private void handleInitEvent(String serviceUnit) { log(null, serviceUnit, null, null); } - private CompletableFuture pubAsync(String serviceUnit, ServiceUnitStateData data) { - CompletableFuture future = new CompletableFuture<>(); - producer.newMessage() - .key(serviceUnit) - .value(data) - .sendAsync() - .whenComplete((messageId, e) -> { + private CompletableFuture pubAsync(String serviceUnit, ServiceUnitStateData data) { + return tableview.put(serviceUnit, data) + .whenComplete((__, e) -> { if (e != null) { log.error("Failed to publish the message: serviceUnit:{}, data:{}", serviceUnit, data, e); - future.completeExceptionally(e); - } else { - future.complete(messageId); } }); - return future; } - private CompletableFuture tombstoneAsync(String serviceUnit) { - return pubAsync(serviceUnit, null); + private CompletableFuture tombstoneAsync(String serviceUnit) { + return tableview.delete(serviceUnit) + .whenComplete((__, e) -> { + if (e != null) { + log.error("Failed to tombstone the serviceUnit:{}}", + serviceUnit, e); + } + }); } private boolean isTargetBroker(String broker) { @@ -943,11 +921,12 @@ private boolean isTargetBroker(String broker) { return broker.equals(brokerId); } + private CompletableFuture deferGetOwner(String serviceUnit) { var future = new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) .exceptionally(e -> { - var ownerAfter = getOwner(serviceUnit); + var ownerAfter = getOwnerNow(serviceUnit); log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " + "return the current owner:{}", brokerId, serviceUnit, ownerAfter, e); @@ -967,7 +946,7 @@ private CompletableFuture dedupeGetOwnerRequest(String serviceUnit) { var requested = new MutableObject>(); try { return getOwnerRequests.computeIfAbsent(serviceUnit, k -> { - var ownerBefore = getOwner(serviceUnit); + var ownerBefore = getOwnerNow(serviceUnit); if (ownerBefore != null && ownerBefore.isPresent()) { // Here, we do the broker active check first with the computeIfAbsent lock requested.setValue(brokerRegistry.lookupAsync(ownerBefore.get()) @@ -1073,7 +1052,6 @@ private CompletableFuture splitServiceUnit(String serviceUnit, ServiceUnit } - @VisibleForTesting protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, NamespaceBundleFactory bundleFactory, @@ -1093,7 +1071,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, .thenAccept(__ -> // Update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(parentBundle)) .thenAccept(__ -> pubAsync(parentBundle.toString(), new ServiceUnitStateData( - Deleted, null, parentData.sourceBroker(), getNextVersionId(parentData)))) + Deleted, null, parentData.sourceBroker(), getNextVersionId(parentData)))) .thenAccept(__ -> { double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime)); log.info("Successfully split {} parent namespace-bundle to {} in {} ms", @@ -1109,7 +1087,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, log.warn("Failed to update bundle range in metadata store. Retrying {} th / {} limit", counter.get(), NamespaceService.BUNDLE_SPLIT_RETRY_LIMIT, ex); pulsar.getExecutor().schedule(() -> splitServiceUnitOnceAndRetry( - namespaceService, bundleFactory, algorithm, parentBundle, childBundles, + namespaceService, bundleFactory, algorithm, parentBundle, childBundles, boundaries, parentData, counter, startTime, completionFuture), 100, MILLISECONDS); } else { @@ -1156,45 +1134,43 @@ private CompletableFuture getSplitNamespaceBundles(NamespaceSe NamespaceBundle parentBundle, List childBundles, List boundaries) { - CompletableFuture future = new CompletableFuture(); final var debug = debug(); - var targetNsBundle = bundleFactory.getBundles(parentBundle.getNamespaceObject()); - boolean found = false; - try { - targetNsBundle.validateBundle(parentBundle); - } catch (IllegalArgumentException e) { - if (debug) { - log.info("Namespace bundles do not contain the parent bundle:{}", - parentBundle); - } - for (var childBundle : childBundles) { - try { - targetNsBundle.validateBundle(childBundle); - if (debug) { - log.info("Namespace bundles contain the child bundle:{}", - childBundle); + return bundleFactory.getBundlesAsync(parentBundle.getNamespaceObject()) + .thenCompose(targetNsBundle -> { + boolean found = false; + try { + targetNsBundle.validateBundle(parentBundle); + } catch (IllegalArgumentException e) { + if (debug) { + log.info("Namespace bundles do not contain the parent bundle:{}", + parentBundle); + } + for (var childBundle : childBundles) { + try { + targetNsBundle.validateBundle(childBundle); + if (debug) { + log.info("Namespace bundles contain the child bundle:{}", + childBundle); + } + } catch (Exception ex) { + throw FutureUtil.wrapToCompletionException( + new BrokerServiceException.ServiceUnitNotReadyException( + "Namespace bundles do not contain the child bundle:" + childBundle, e)); + } + } + found = true; + } catch (Exception e) { + throw FutureUtil.wrapToCompletionException( + new BrokerServiceException.ServiceUnitNotReadyException( + "Failed to validate the parent bundle in the namespace bundles.", e)); } - } catch (Exception ex) { - future.completeExceptionally( - new BrokerServiceException.ServiceUnitNotReadyException( - "Namespace bundles do not contain the child bundle:" + childBundle, e)); - return future; - } - } - found = true; - } catch (Exception e) { - future.completeExceptionally( - new BrokerServiceException.ServiceUnitNotReadyException( - "Failed to validate the parent bundle in the namespace bundles.", e)); - return future; - } - if (found) { - future.complete(targetNsBundle); - return future; - } else { - return namespaceService.getSplitBoundary(parentBundle, algorithm, boundaries) - .thenApply(splitBundlesPair -> splitBundlesPair.getLeft()); - } + if (found) { + return CompletableFuture.completedFuture(targetNsBundle); + } else { + return namespaceService.getSplitBoundary(parentBundle, algorithm, boundaries) + .thenApply(splitBundlesPair -> splitBundlesPair.getLeft()); + } + }); } private CompletableFuture updateSplitNamespaceBundlesAsync( @@ -1215,7 +1191,12 @@ private CompletableFuture updateSplitNamespaceBundlesAsync( }); } - public void handleMetadataSessionEvent(SessionEvent e) { + /** + * The stability of the metadata connection is important + * to determine how to handle the broker deletion(unavailable) event notified from the metadata store. + */ + @VisibleForTesting + protected void handleMetadataSessionEvent(SessionEvent e) { if (e == SessionReestablished || e == SessionLost) { lastMetadataSessionEvent = e; lastMetadataSessionEventTimestamp = System.currentTimeMillis(); @@ -1224,7 +1205,30 @@ public void handleMetadataSessionEvent(SessionEvent e) { } } - public void handleBrokerRegistrationEvent(String broker, NotificationType type) { + /** + * Case 1: If NotificationType is Deleted, + * it will schedule a clean-up operation to release the ownerships of the deleted broker. + * + * Sub-case1: If the metadata connection has been stable for long time, + * it will immediately execute the cleanup operation to guarantee high-availability. + * + * Sub-case2: If the metadata connection has been stable only for short time, + * it will defer the clean-up operation for some time and execute it. + * This is to gracefully handle the case when metadata connection is flaky -- + * If the deleted broker comes back very soon, + * we better cancel the clean-up operation for high-availability. + * + * Sub-case3: If the metadata connection is unstable, + * it will not schedule the clean-up operation, as the broker-metadata connection is lost. + * The brokers will continue to serve existing topics connections, + * and we better not to interrupt the existing topic connections for high-availability. + * + * + * Case 2: If NotificationType is Created, + * it will cancel any scheduled clean-up operation if still not executed. + */ + @VisibleForTesting + protected void handleBrokerRegistrationEvent(String broker, NotificationType type) { if (type == NotificationType.Created) { log.info("BrokerRegistry detected the broker:{} registry has been created.", broker); handleBrokerCreationEvent(broker); @@ -1263,8 +1267,13 @@ private void handleBrokerCreationEvent(String broker) { } private void handleBrokerDeletionEvent(String broker) { - if (!isChannelOwner()) { - log.warn("This broker is not the leader now. Ignoring BrokerDeletionEvent for broker {}.", broker); + try { + if (!isChannelOwner()) { + log.warn("This broker is not the leader now. Ignoring BrokerDeletionEvent for broker {}.", broker); + return; + } + } catch (Exception e) { + log.error("Failed to handle broker deletion event.", e); return; } MetadataState state = getMetadataState(); @@ -1291,7 +1300,7 @@ private void scheduleCleanup(String broker, long delayInSecs) { var future = CompletableFuture .runAsync(() -> { try { - doCleanup(broker); + doCleanup(broker, false); } catch (Throwable e) { log.error("Failed to run the cleanup job for the broker {}, " + "totalCleanupErrorCnt:{}.", @@ -1316,7 +1325,9 @@ private void scheduleCleanup(String broker, long delayInSecs) { } - private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) { + private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker, + boolean gracefully) { + final var version = getNextVersionId(orphanData); try { selectBroker(serviceUnit, inactiveBroker) @@ -1330,9 +1341,14 @@ private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanDa } else if (orphanData.state() == Owned) { // if Owned, set orphan.dstBroker() as source to clean it up in case it is still // alive. - return new ServiceUnitStateData(Owned, selectedBroker, - selectedBroker.equals(orphanData.dstBroker()) ? null : - orphanData.dstBroker(), + var sourceBroker = selectedBroker.equals(orphanData.dstBroker()) ? null : + orphanData.dstBroker(); + // if gracefully, try to release ownership first + var overrideState = gracefully && sourceBroker != null ? Releasing : Owned; + return new ServiceUnitStateData( + overrideState, + selectedBroker, + sourceBroker, true, version); } else { // if Assigning or Releasing, set orphan.sourceBroker() as source @@ -1395,7 +1411,7 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max System.currentTimeMillis() - started); } - private synchronized void doCleanup(String broker) { + private synchronized void doCleanup(String broker, boolean gracefully) { try { if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) .isEmpty()) { @@ -1422,14 +1438,14 @@ private synchronized void doCleanup(String broker) { if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) { orphanSystemServiceUnits.put(serviceUnit, stateData); } else { - overrideOwnership(serviceUnit, stateData, broker); + overrideOwnership(serviceUnit, stateData, broker, gracefully); } orphanServiceUnitCleanupCnt++; } } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight non-system bundle override messages.", e); } @@ -1449,11 +1465,11 @@ private synchronized void doCleanup(String broker) { // clean system bundles in the end for (var orphanSystemServiceUnit : orphanSystemServiceUnits.entrySet()) { log.info("Overriding orphan system service unit:{}", orphanSystemServiceUnit.getKey()); - overrideOwnership(orphanSystemServiceUnit.getKey(), orphanSystemServiceUnit.getValue(), broker); + overrideOwnership(orphanSystemServiceUnit.getKey(), orphanSystemServiceUnit.getValue(), broker, gracefully); } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight system bundle override messages.", e); } @@ -1486,8 +1502,13 @@ private CompletableFuture> selectBroker(String serviceUnit, Str @VisibleForTesting protected void monitorOwnerships(List brokers) { - if (!isChannelOwner()) { - log.warn("This broker is not the leader now. Skipping ownership monitor."); + try { + if (!isChannelOwner()) { + log.warn("This broker is not the leader now. Skipping ownership monitor."); + return; + } + } catch (Exception e) { + log.error("Failed to monitor ownerships", e); return; } @@ -1571,13 +1592,13 @@ protected void monitorOwnerships(List brokers) { for (var etr : timedOutInFlightStateServiceUnits.entrySet()) { var orphanServiceUnit = etr.getKey(); var orphanData = etr.getValue(); - overrideOwnership(orphanServiceUnit, orphanData, null); + overrideOwnership(orphanServiceUnit, orphanData, null, false); orphanServiceUnitCleanupCnt++; } } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight messages.", e); } @@ -1638,10 +1659,8 @@ private int getTotalOwnedServiceUnitCnt() { if (lastOwnEventHandledAt > lastOwnedServiceUnitCountAt || now - lastOwnedServiceUnitCountAt > MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS) { int cnt = 0; - for (var data : tableview.values()) { - if (data.state() == Owned && isTargetBroker(data.dstBroker())) { - cnt++; - } + for (var e : tableview.ownedServiceUnits()) { + cnt++; } lastOwnedServiceUnitCountAt = now; totalOwnedServiceUnitCnt = cnt; @@ -1787,7 +1806,25 @@ public Set> getOwnershipEntrySet() { return tableview.entrySet(); } + @Override + public Set getOwnedServiceUnits() { + if (!validateChannelState(Started, true)) { + throw new IllegalStateException("Invalid channel state:" + channelState.name()); + } + return tableview.ownedServiceUnits(); + } + public static ServiceUnitStateChannel get(PulsarService pulsar) { return ExtensibleLoadManagerImpl.get(pulsar.getLoadManager().get()).getServiceUnitStateChannel(); } + + @VisibleForTesting + protected void disable() { + channelState = Disabled; + } + + @VisibleForTesting + protected void enable() { + channelState = Started; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java index 307d3a4acb175..e85134e611632 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java @@ -75,4 +75,19 @@ public ServiceUnitStateData(ServiceUnitState state, String dstBroker, boolean fo public static ServiceUnitState state(ServiceUnitStateData data) { return data == null ? ServiceUnitState.Init : data.state(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ServiceUnitStateData that = (ServiceUnitStateData) o; + + return versionId == that.versionId; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java similarity index 82% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java index 6a98b79be81d0..b1dbb6fac8709 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java @@ -20,21 +20,27 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import com.google.common.annotations.VisibleForTesting; import java.util.function.BiConsumer; +import lombok.Setter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.topics.TopicCompactionStrategy; -public class ServiceUnitStateCompactionStrategy implements TopicCompactionStrategy { +public class ServiceUnitStateDataConflictResolver implements TopicCompactionStrategy { private final Schema schema; private BiConsumer skippedMsgHandler; private boolean checkBrokers = true; - public ServiceUnitStateCompactionStrategy() { + @Setter + private ServiceUnitState.StorageType storageType = SystemTopic; + + public ServiceUnitStateDataConflictResolver() { schema = Schema.JSON(ServiceUnitStateData.class); } @@ -70,8 +76,16 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to } else if (from.versionId() >= to.versionId()) { return true; } else if (from.versionId() < to.versionId() - 1) { // Compacted - return false; + // If the system topic is compacted, to.versionId can be bigger than from.versionId by 2 or more. + // e.g. (Owned, v1) -> (Owned, v3) + return storageType != SystemTopic; } // else from.versionId() == to.versionId() - 1 // continue to check further + } else { + // If `from` is null, to.versionId should start at 1 over metadata store. + // In this case, to.versionId can be bigger than 1 over the system topic, if compacted. + if (storageType == MetadataStore) { + return to.versionId() != 1; + } } if (to.force()) { @@ -80,7 +94,7 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to ServiceUnitState prevState = state(from); ServiceUnitState state = state(to); - if (!ServiceUnitState.isValidTransition(prevState, state)) { + if (!ServiceUnitState.isValidTransition(prevState, state, storageType)) { return true; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java new file mode 100644 index 0000000000000..f488b31c77415 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java @@ -0,0 +1,155 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; +import org.apache.pulsar.metadata.tableview.impl.MetadataStoreTableViewImpl; + +@Slf4j +public class ServiceUnitStateMetadataStoreTableViewImpl extends ServiceUnitStateTableViewBase { + public static final String PATH_PREFIX = "/service_unit_state"; + private static final String VALID_PATH_REG_EX = "^\\/service_unit_state\\/.*\\/0x[0-9a-fA-F]{8}_0x[0-9a-fA-F]{8}$"; + private static final Pattern VALID_PATH_PATTERN; + + static { + try { + VALID_PATH_PATTERN = Pattern.compile(VALID_PATH_REG_EX); + } catch (PatternSyntaxException error) { + log.error("Invalid regular expression {}", VALID_PATH_REG_EX, error); + throw new IllegalArgumentException(error); + } + } + private ServiceUnitStateDataConflictResolver conflictResolver; + private volatile MetadataStoreTableView tableview; + + public void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) + throws MetadataStoreException { + init(pulsar); + conflictResolver = new ServiceUnitStateDataConflictResolver(); + conflictResolver.setStorageType(MetadataStore); + tableview = new MetadataStoreTableViewImpl<>(ServiceUnitStateData.class, + pulsar.getBrokerId(), + pulsar.getLocalMetadataStore(), + PATH_PREFIX, + this::resolveConflict, + this::validateServiceUnitPath, + List.of(this::updateOwnedServiceUnits, tailItemListener), + List.of(this::updateOwnedServiceUnits, existingItemListener), + TimeUnit.SECONDS.toMillis(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds()) + ); + tableview.start(); + + } + + protected boolean resolveConflict(ServiceUnitStateData prev, ServiceUnitStateData cur) { + return !conflictResolver.shouldKeepLeft(prev, cur); + } + + + protected boolean validateServiceUnitPath(String path) { + try { + var matcher = VALID_PATH_PATTERN.matcher(path); + return matcher.matches(); + } catch (Exception e) { + return false; + } + } + + + @Override + public void close() throws IOException { + if (tableview != null) { + tableview = null; + log.info("Successfully closed the channel tableview."); + } + } + + private boolean isValidState() { + if (tableview == null) { + return false; + } + return true; + } + + @Override + public ServiceUnitStateData get(String key) { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.get(key); + } + + @Override + public CompletableFuture put(String key, @NonNull ServiceUnitStateData value) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + return tableview.put(key, value).exceptionally(e -> { + if (e.getCause() instanceof MetadataStoreTableView.ConflictException) { + return null; + } + throw FutureUtil.wrapToCompletionException(e); + }); + } + + @Override + public void flush(long waitDurationInMillis) { + // no-op + } + + @Override + public CompletableFuture delete(String key) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + return tableview.delete(key).exceptionally(e -> { + if (e.getCause() instanceof MetadataStoreException.NotFoundException) { + return null; + } + throw FutureUtil.wrapToCompletionException(e); + }); + } + + + @Override + public Set> entrySet() { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.entrySet(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java new file mode 100644 index 0000000000000..5ac57fe5c19c6 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java @@ -0,0 +1,113 @@ +/* + * 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.loadbalance.extensions.channel; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.naming.NamespaceBundle; + +/** + * Given that the ServiceUnitStateChannel event-sources service unit (bundle) ownership states via a persistent store + * and reacts to ownership changes, the ServiceUnitStateTableView provides an interface to the + * ServiceUnitStateChannel's persistent store and its locally replicated ownership view (tableview) with listener + * registration. It initially populates its local table view by scanning existing items in the remote store. The + * ServiceUnitStateTableView receives notifications whenever ownership states are updated in the remote store, and + * upon notification, it applies the updates to its local tableview with the listener logic. + */ +public interface ServiceUnitStateTableView extends Closeable { + + /** + * Starts the tableview. + * It initially populates its local table view by scanning existing items in the remote store, and it starts + * listening to service unit ownership changes from the remote store. + * @param pulsar pulsar service reference + * @param tailItemListener listener to listen tail(newly updated) items + * @param existingItemListener listener to listen existing items + * @throws IOException if it fails to init the tableview. + */ + void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) throws IOException; + + + /** + * Closes the tableview. + * @throws IOException if it fails to close the tableview. + */ + void close() throws IOException; + + /** + * Gets one item from the local tableview. + * @param key the key to get + * @return value if exists. Otherwise, null. + */ + ServiceUnitStateData get(String key); + + /** + * Tries to put the item in the persistent store. + * If it completes, all peer tableviews (including the local one) will be notified and be eventually consistent + * with this put value. + * + * It ignores put operation if the input value conflicts with the existing one in the persistent store. + * + * @param key the key to put + * @param value the value to put + * @return a future to track the completion of the operation + */ + CompletableFuture put(String key, ServiceUnitStateData value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + * + * It ignores delete operation if the key is not present in the persistent store. + * + * @param key the key to delete + * @return a future to track the completion of the operation + */ + CompletableFuture delete(String key); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); + + /** + * Returns service units (namespace bundles) owned by this broker. + * @return a set of owned service units (namespace bundles) + */ + Set ownedServiceUnits(); + + /** + * Tries to flush any batched or buffered updates. + * @param waitDurationInMillis time to wait until complete. + * @throws ExecutionException + * @throws InterruptedException + * @throws TimeoutException + */ + void flush(long waitDurationInMillis) throws ExecutionException, InterruptedException, TimeoutException; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java new file mode 100644 index 0000000000000..b690ef101e168 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java @@ -0,0 +1,92 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; +import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.metadata.api.MetadataStoreException; + +/** + * ServiceUnitStateTableView base class. + */ +@Slf4j +abstract class ServiceUnitStateTableViewBase implements ServiceUnitStateTableView { + protected static final String INVALID_STATE_ERROR_MSG = "The tableview has not been started."; + private final Map ownedServiceUnitsMap = new ConcurrentHashMap<>(); + private final Set ownedServiceUnits = Collections.unmodifiableSet(ownedServiceUnitsMap.keySet()); + private String brokerId; + private PulsarService pulsar; + protected void init(PulsarService pulsar) throws MetadataStoreException { + this.pulsar = pulsar; + this.brokerId = pulsar.getBrokerId(); + // Add heartbeat and SLA monitor namespace bundle. + NamespaceName heartbeatNamespace = + NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); + NamespaceName heartbeatNamespaceV2 = NamespaceService + .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); + NamespaceName slaMonitorNamespace = NamespaceService + .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); + try { + pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespace) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespaceV2)) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(slaMonitorNamespace)) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenApply(__ -> null).get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), + TimeUnit.SECONDS); + } catch (Exception e) { + throw new MetadataStoreException(e); + } + } + + @Override + public Set ownedServiceUnits() { + return ownedServiceUnits; + } + + protected void updateOwnedServiceUnits(String key, ServiceUnitStateData val) { + NamespaceBundle namespaceBundle = LoadManagerShared.getNamespaceBundle(pulsar, key); + var state = ServiceUnitStateData.state(val); + ownedServiceUnitsMap.compute(namespaceBundle, (k, v) -> { + if (state == Owned && brokerId.equals(val.dstBroker())) { + return true; + } else if (state == Splitting && brokerId.equals(val.sourceBroker())) { + return true; + } else { + return null; + } + }); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java new file mode 100644 index 0000000000000..8dfaddcdabca1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java @@ -0,0 +1,177 @@ +/* + * 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.loadbalance.extensions.channel; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; + +@Slf4j +public class ServiceUnitStateTableViewImpl extends ServiceUnitStateTableViewBase { + + public static final String TOPIC = TopicName.get( + TopicDomain.persistent.value(), + SYSTEM_NAMESPACE, + "loadbalancer-service-unit-state").toString(); + private static final int MAX_OUTSTANDING_PUB_MESSAGES = 500; + public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; + private volatile Producer producer; + private volatile TableView tableview; + + public void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) throws IOException { + boolean debug = ExtensibleLoadManagerImpl.debug(pulsar.getConfiguration(), log); + + init(pulsar); + + var schema = Schema.JSON(ServiceUnitStateData.class); + + ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); + + if (producer != null) { + producer.close(); + if (debug) { + log.info("Closed the channel producer."); + } + } + + producer = pulsar.getClient().newProducer(schema) + .enableBatching(true) + .compressionType(MSG_COMPRESSION_TYPE) + .maxPendingMessages(MAX_OUTSTANDING_PUB_MESSAGES) + .blockIfQueueFull(true) + .topic(TOPIC) + .create(); + + if (debug) { + log.info("Successfully started the channel producer."); + } + + if (tableview != null) { + tableview.close(); + if (debug) { + log.info("Closed the channel tableview."); + } + } + + tableview = pulsar.getClient().newTableViewBuilder(schema) + .topic(TOPIC) + .loadConf(Map.of( + "topicCompactionStrategyClassName", + ServiceUnitStateDataConflictResolver.class.getName())) + .create(); + tableview.listen(this::updateOwnedServiceUnits); + tableview.listen(tailItemListener); + tableview.forEach(this::updateOwnedServiceUnits); + tableview.forEach(existingItemListener); + + } + + private boolean isValidState() { + if (tableview == null || producer == null) { + return false; + } + return true; + } + + + @Override + public void close() throws IOException { + + if (tableview != null) { + tableview.close(); + tableview = null; + log.info("Successfully closed the channel tableview."); + } + + if (producer != null) { + producer.close(); + producer = null; + log.info("Successfully closed the channel producer."); + } + } + + @Override + public ServiceUnitStateData get(String key) { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.get(key); + } + + @Override + public CompletableFuture put(String key, ServiceUnitStateData value) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + CompletableFuture future = new CompletableFuture<>(); + producer.newMessage() + .key(key) + .value(value) + .sendAsync() + .whenComplete((messageId, e) -> { + if (e != null) { + log.error("Failed to publish the message: serviceUnit:{}, data:{}", + key, value, e); + future.completeExceptionally(e); + } else { + future.complete(null); + } + }); + return future; + } + + @Override + public void flush(long waitDurationInMillis) throws InterruptedException, TimeoutException, ExecutionException { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + producer.flushAsync().get(waitDurationInMillis, MILLISECONDS); + } + + @Override + public CompletableFuture delete(String key) { + return put(key, null); + } + + @Override + public Set> entrySet() { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.entrySet(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java new file mode 100644 index 0000000000000..10ab39a66d279 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java @@ -0,0 +1,281 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.ServiceConfiguration.ServiceUnitTableViewSyncerType.SystemTopicToMetadataStoreSyncer; +import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.COMPACTION_THRESHOLD; +import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.configureSystemTopics; +import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; + +/** + * Defines ServiceUnitTableViewSyncer. + * It syncs service unit(bundle) states between metadata store and system topic table views. + * One could enable this syncer before migration from one to the other and disable it after the migration finishes. + */ +@Slf4j +public class ServiceUnitStateTableViewSyncer implements Closeable { + private static final int MAX_CONCURRENT_SYNC_COUNT = 100; + private static final int SYNC_WAIT_TIME_IN_SECS = 300; + private PulsarService pulsar; + private volatile ServiceUnitStateTableView systemTopicTableView; + private volatile ServiceUnitStateTableView metadataStoreTableView; + private volatile boolean isActive = false; + + + public void start(PulsarService pulsar) + throws IOException, TimeoutException, InterruptedException, ExecutionException { + if (!pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + return; + } + + if (isActive) { + return; + } + this.pulsar = pulsar; + + try { + + syncExistingItems(); + // disable compaction + if (!configureSystemTopics(pulsar, 0)) { + throw new IllegalStateException("Failed to disable compaction"); + } + syncTailItems(); + + isActive = true; + + } catch (Throwable e) { + log.error("Failed to start ServiceUnitStateTableViewSyncer", e); + throw e; + } + } + + private CompletableFuture syncToSystemTopic(String key, ServiceUnitStateData data) { + return systemTopicTableView.put(key, data); + } + + private CompletableFuture syncToMetadataStore(String key, ServiceUnitStateData data) { + return metadataStoreTableView.put(key, data); + } + + private void dummy(String key, ServiceUnitStateData data) { + } + + private void syncExistingItems() + throws IOException, ExecutionException, InterruptedException, TimeoutException { + long started = System.currentTimeMillis(); + @Cleanup + ServiceUnitStateTableView metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); + metadataStoreTableView.start( + pulsar, + this::dummy, + this::dummy + ); + + @Cleanup + ServiceUnitStateTableView systemTopicTableView = new ServiceUnitStateTableViewImpl(); + systemTopicTableView.start( + pulsar, + this::dummy, + this::dummy + ); + + + var syncer = pulsar.getConfiguration().getLoadBalancerServiceUnitTableViewSyncer(); + if (syncer == SystemTopicToMetadataStoreSyncer) { + clean(metadataStoreTableView); + syncExistingItemsToMetadataStore(systemTopicTableView); + } else { + clean(systemTopicTableView); + syncExistingItemsToSystemTopic(metadataStoreTableView, systemTopicTableView); + } + + if (!waitUntilSynced(metadataStoreTableView, systemTopicTableView, started)) { + throw new TimeoutException( + syncer + " failed to sync existing items in tableviews. MetadataStoreTableView.size: " + + metadataStoreTableView.entrySet().size() + + ", SystemTopicTableView.size: " + systemTopicTableView.entrySet().size() + " in " + + SYNC_WAIT_TIME_IN_SECS + " secs"); + } + + log.info("Synced existing items MetadataStoreTableView.size:{} , " + + "SystemTopicTableView.size: {} in {} secs", + metadataStoreTableView.entrySet().size(), systemTopicTableView.entrySet().size(), + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started)); + } + + private void syncTailItems() throws InterruptedException, IOException, TimeoutException { + long started = System.currentTimeMillis(); + + if (metadataStoreTableView != null) { + metadataStoreTableView.close(); + metadataStoreTableView = null; + } + + if (systemTopicTableView != null) { + systemTopicTableView.close(); + systemTopicTableView = null; + } + + this.metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); + this.metadataStoreTableView.start( + pulsar, + this::syncToSystemTopic, + this::dummy + ); + log.info("Started MetadataStoreTableView"); + + this.systemTopicTableView = new ServiceUnitStateTableViewImpl(); + this.systemTopicTableView.start( + pulsar, + this::syncToMetadataStore, + this::dummy + ); + log.info("Started SystemTopicTableView"); + + var syncer = pulsar.getConfiguration().getLoadBalancerServiceUnitTableViewSyncer(); + if (!waitUntilSynced(metadataStoreTableView, systemTopicTableView, started)) { + throw new TimeoutException( + syncer + " failed to sync tableviews. MetadataStoreTableView.size: " + + metadataStoreTableView.entrySet().size() + + ", SystemTopicTableView.size: " + systemTopicTableView.entrySet().size() + " in " + + SYNC_WAIT_TIME_IN_SECS + " secs"); + } + + + log.info("Successfully started ServiceUnitStateTableViewSyncer MetadataStoreTableView.size:{} , " + + "SystemTopicTableView.size: {} in {} secs", + metadataStoreTableView.entrySet().size(), systemTopicTableView.entrySet().size(), + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started)); + } + + private void syncExistingItemsToMetadataStore(ServiceUnitStateTableView src) + throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { + // Directly use store to sync existing items to metadataStoreTableView(otherwise, they are conflicted out) + var store = pulsar.getLocalMetadataStore(); + var writer = ObjectMapperFactory.getMapper().writer(); + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + List> futures = new ArrayList<>(); + var srcIter = src.entrySet().iterator(); + while (srcIter.hasNext()) { + var e = srcIter.next(); + futures.add(store.put(ServiceUnitStateMetadataStoreTableViewImpl.PATH_PREFIX + "/" + e.getKey(), + writer.writeValueAsBytes(e.getValue()), Optional.empty()).thenApply(__ -> null)); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !srcIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private void syncExistingItemsToSystemTopic(ServiceUnitStateTableView src, + ServiceUnitStateTableView dst) + throws ExecutionException, InterruptedException, TimeoutException { + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + List> futures = new ArrayList<>(); + var srcIter = src.entrySet().iterator(); + while (srcIter.hasNext()) { + var e = srcIter.next(); + futures.add(dst.put(e.getKey(), e.getValue())); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !srcIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private void clean(ServiceUnitStateTableView dst) + throws ExecutionException, InterruptedException, TimeoutException { + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + var dstIter = dst.entrySet().iterator(); + List> futures = new ArrayList<>(); + while (dstIter.hasNext()) { + var e = dstIter.next(); + futures.add(dst.delete(e.getKey())); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !dstIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private boolean waitUntilSynced(ServiceUnitStateTableView srt, ServiceUnitStateTableView dst, long started) + throws InterruptedException { + while (srt.entrySet().size() != dst.entrySet().size()) { + if (TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started) + > SYNC_WAIT_TIME_IN_SECS) { + return false; + } + Thread.sleep(100); + } + return true; + } + + @Override + public void close() throws IOException { + if (!isActive) { + return; + } + + if (!configureSystemTopics(pulsar, COMPACTION_THRESHOLD)) { + throw new IllegalStateException("Failed to enable compaction"); + } + + try { + if (systemTopicTableView != null) { + systemTopicTableView.close(); + systemTopicTableView = null; + log.info("Closed SystemTopicTableView"); + } + } catch (Exception e) { + log.error("Failed to close SystemTopicTableView", e); + throw e; + } + + try { + if (metadataStoreTableView != null) { + metadataStoreTableView.close(); + metadataStoreTableView = null; + log.info("Closed MetadataStoreTableView"); + } + } catch (Exception e) { + log.error("Failed to close MetadataStoreTableView", e); + throw e; + } + + log.info("Successfully closed ServiceUnitStateTableViewSyncer."); + isActive = false; + } + + public boolean isActive() { + return isActive; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java index a7deeeaad8a5c..8096d1908b928 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java @@ -103,4 +103,10 @@ public interface LoadDataStore extends Closeable { */ void startProducer() throws LoadDataStoreException; + /** + * Shutdowns the data store. + */ + default void shutdown() throws IOException { + close(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index e9289d3ccdac2..c9d18676cfa99 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -43,20 +43,17 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2; + private static final String SHUTDOWN_ERR_MSG = "This load store tableview has been shutdown"; private static final long INIT_TIMEOUT_IN_SECS = 5; - private volatile TableView tableView; private volatile long tableViewLastUpdateTimestamp; - + private volatile long producerLastPublishTimestamp; private volatile Producer producer; - private final ServiceConfiguration conf; - private final PulsarClient client; - private final String topic; - private final Class clazz; + private volatile boolean isShutdown; public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class clazz) throws LoadDataStoreException { @@ -65,6 +62,7 @@ public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class c this.client = pulsar.getClient(); this.topic = topic; this.clazz = clazz; + this.isShutdown = false; } catch (Exception e) { throw new LoadDataStoreException(e); } @@ -72,41 +70,76 @@ public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class c @Override public synchronized CompletableFuture pushAsync(String key, T loadData) { - validateProducer(); - return producer.newMessage().key(key).value(loadData).sendAsync().thenAccept(__ -> {}); + String msg = validateProducer(); + if (StringUtils.isNotBlank(msg)) { + return CompletableFuture.failedFuture(new IllegalStateException(msg)); + } + return producer.newMessage().key(key).value(loadData).sendAsync() + .thenAccept(__ -> producerLastPublishTimestamp = System.currentTimeMillis()); } @Override public synchronized CompletableFuture removeAsync(String key) { - validateProducer(); - return producer.newMessage().key(key).value(null).sendAsync().thenAccept(__ -> {}); + String msg = validateProducer(); + if (StringUtils.isNotBlank(msg)) { + return CompletableFuture.failedFuture(new IllegalStateException(msg)); + } + return producer.newMessage().key(key).value(null).sendAsync() + .thenAccept(__ -> producerLastPublishTimestamp = System.currentTimeMillis()); } @Override public synchronized Optional get(String key) { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return Optional.ofNullable(tableView.get(key)); } @Override public synchronized void forEach(BiConsumer action) { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } tableView.forEach(action); } public synchronized Set> entrySet() { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return tableView.entrySet(); } @Override public synchronized int size() { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return tableView.size(); } + private void validateState() { + if (isShutdown) { + throw new IllegalStateException(SHUTDOWN_ERR_MSG); + } + } + + + @Override + public synchronized void init() throws IOException { + validateState(); + close(); + start(); + } + @Override public synchronized void closeTableView() throws IOException { + validateState(); if (tableView != null) { tableView.close(); tableView = null; @@ -115,16 +148,26 @@ public synchronized void closeTableView() throws IOException { @Override public synchronized void start() throws LoadDataStoreException { + validateState(); startProducer(); startTableView(); } + private synchronized void closeProducer() throws IOException { + validateState(); + if (producer != null) { + producer.close(); + producer = null; + } + } @Override public synchronized void startTableView() throws LoadDataStoreException { + validateState(); if (tableView == null) { try { tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).createAsync() .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + tableViewLastUpdateTimestamp = System.currentTimeMillis(); tableView.forEachAndListen((k, v) -> tableViewLastUpdateTimestamp = System.currentTimeMillis()); } catch (Exception e) { @@ -133,13 +176,14 @@ public synchronized void startTableView() throws LoadDataStoreException { } } } - @Override public synchronized void startProducer() throws LoadDataStoreException { + validateState(); if (producer == null) { try { producer = client.newProducer(Schema.JSON(clazz)).topic(topic).createAsync() .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + producerLastPublishTimestamp = System.currentTimeMillis(); } catch (Exception e) { producer = null; throw new LoadDataStoreException(e); @@ -149,38 +193,63 @@ public synchronized void startProducer() throws LoadDataStoreException { @Override public synchronized void close() throws IOException { - if (producer != null) { - producer.close(); - producer = null; - } + validateState(); + closeProducer(); closeTableView(); } @Override - public synchronized void init() throws IOException { + public synchronized void shutdown() throws IOException { close(); - start(); + isShutdown = true; } - private void validateProducer() { - if (producer == null) { + private String validateProducer() { + if (isShutdown) { + return SHUTDOWN_ERR_MSG; + } + String restartReason = getRestartReason(producer, producerLastPublishTimestamp); + if (StringUtils.isNotBlank(restartReason)) { try { + closeProducer(); startProducer(); - log.info("Restarted producer on {}", topic); + log.info("Restarted producer on {}, {}", topic, restartReason); } catch (Exception e) { - log.error("Failed to restart producer on {}", topic, e); - throw new RuntimeException(e); + String msg = "Failed to restart producer on " + topic + ", restart reason: " + restartReason; + log.error(msg, e); + return msg; } } + return null; } - private void validateTableView() { + private String validateTableView() { + if (isShutdown) { + return SHUTDOWN_ERR_MSG; + } + String restartReason = getRestartReason(tableView, tableViewLastUpdateTimestamp); + if (StringUtils.isNotBlank(restartReason)) { + try { + closeTableView(); + startTableView(); + log.info("Restarted tableview on {}, {}", topic, restartReason); + } catch (Exception e) { + String msg = "Failed to tableview on " + topic + ", restart reason: " + restartReason; + log.error(msg, e); + return msg; + } + } + return null; + } + + private String getRestartReason(Object obj, long lastUpdateTimestamp) { + String restartReason = null; - if (tableView == null) { - restartReason = "table view is null"; + if (obj == null) { + restartReason = "object is null"; } else { - long inactiveDuration = System.currentTimeMillis() - tableViewLastUpdateTimestamp; + long inactiveDuration = System.currentTimeMillis() - lastUpdateTimestamp; long threshold = TimeUnit.MINUTES.toMillis(conf.getLoadBalancerReportUpdateMaxIntervalMinutes()) * LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART; if (inactiveDuration > threshold) { @@ -189,17 +258,6 @@ private void validateTableView() { TimeUnit.MILLISECONDS.toSeconds(threshold)); } } - - if (StringUtils.isNotBlank(restartReason)) { - tableViewLastUpdateTimestamp = 0; - try { - closeTableView(); - startTableView(); - log.info("Restarted tableview on {}, {}", topic, restartReason); - } catch (Exception e) { - log.error("Failed to restart tableview on {}", topic, e); - throw new RuntimeException(e); - } - } + return restartReason; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 9c0bdc120c474..d664d6812adaa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -21,6 +21,7 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter.isSubscribeRateEnabled; import static org.apache.pulsar.common.naming.SystemTopicNames.isEventSystemTopic; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; @@ -91,8 +92,7 @@ import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateDataConflictResolver; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources; import org.apache.pulsar.broker.service.AbstractReplicator; @@ -247,8 +247,8 @@ public static boolean isDedupCursorName(String name) { // TODO: Create compaction strategy from topic policy when exposing strategic compaction to users. private static Map strategicCompactionMap = Map.of( - ServiceUnitStateChannelImpl.TOPIC, - new ServiceUnitStateCompactionStrategy()); + TOPIC, + new ServiceUnitStateDataConflictResolver()); private CompletableFuture currentOffload = CompletableFuture.completedFuture( (MessageIdImpl) MessageId.earliest); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index e9fafa9c30317..bb224cdf7c40e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -22,26 +22,35 @@ import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; import com.google.common.io.Resources; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.FutureUtil; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServiceBaseTest { @@ -67,8 +76,21 @@ public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServ protected LookupService lookupService; - protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { + protected String serviceUnitStateTableViewClassName; + + protected ArrayList clients = new ArrayList<>(); + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {ServiceUnitStateTableViewImpl.class.getName()}, + {ServiceUnitStateMetadataStoreTableViewImpl.class.getName()} + }; + } + + protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace, String serviceUnitStateTableViewClassName) { this.defaultTestNamespace = defaultTestNamespace; + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; } @Override @@ -82,6 +104,8 @@ protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + conf.setLoadBalancerReportUpdateMaxIntervalMinutes(1); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); conf.setWebServicePortTls(Optional.of(0)); @@ -117,20 +141,44 @@ protected void setup() throws Exception { admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, Sets.newHashSet(this.conf.getClusterName())); lookupService = (LookupService) FieldUtils.readDeclaredField(pulsarClient, "lookup", true); + + for (int i = 0; i < 4; i++) { + clients.add(pulsarClient(lookupUrl.toString(), 100)); + } } + private static PulsarClient pulsarClient(String url, int intervalInMillis) throws PulsarClientException { + return + PulsarClient.builder() + .serviceUrl(url) + .statsInterval(intervalInMillis, TimeUnit.MILLISECONDS).build(); + } + + @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { + List> futures = new ArrayList<>(); + for (PulsarClient client : clients) { + futures.add(client.closeAsync()); + } + futures.add(pulsar2.closeAsync()); + if (additionalPulsarTestContext != null) { additionalPulsarTestContext.close(); additionalPulsarTestContext = null; } super.internalCleanup(); + try { + FutureUtil.waitForAll(futures).join(); + } catch (Throwable e) { + // skip error + } pulsar1 = pulsar2 = null; primaryLoadManager = secondaryLoadManager = null; channel1 = channel2 = null; lookupService = null; + } @BeforeMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 51966f420bf25..4f6a006918318 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -18,8 +18,7 @@ */ package org.apache.pulsar.broker.loadbalance.extensions; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelTest.overrideTableView; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Bandwidth; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.MsgRate; @@ -44,7 +43,6 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -91,7 +89,8 @@ import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -102,6 +101,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter; import org.apache.pulsar.broker.loadbalance.extensions.reporter.BrokerLoadDataReporter; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; +import org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.broker.namespace.LookupOptions; @@ -138,6 +138,7 @@ import org.awaitility.Awaitility; import org.testng.AssertJUnit; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; /** @@ -148,19 +149,20 @@ @SuppressWarnings("unchecked") public class ExtensibleLoadManagerImplTest extends ExtensibleLoadManagerImplBaseTest { - public ExtensibleLoadManagerImplTest() { - super("public/test"); + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplTest(String serviceUnitStateTableViewClassName) { + super("public/test", serviceUnitStateTableViewClassName); } @Test public void testAssignInternalTopic() throws Exception { Optional brokerLookupData1 = primaryLoadManager.assign( - Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + Optional.of(TopicName.get(TOPIC)), + getBundleAsync(pulsar1, TopicName.get(TOPIC)).get(), LookupOptions.builder().build()).get(); Optional brokerLookupData2 = secondaryLoadManager.assign( - Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + Optional.of(TopicName.get(TOPIC)), + getBundleAsync(pulsar1, TopicName.get(TOPIC)).get(), LookupOptions.builder().build()).get(); assertEquals(brokerLookupData1, brokerLookupData2); assertTrue(brokerLookupData1.isPresent()); @@ -320,7 +322,7 @@ public void testUnloadUponTopicLookupFailure() throws Exception { try { pulsar1.getBrokerService().getTopics().put(topicName.toString(), future1); pulsar2.getBrokerService().getTopics().put(topicName.toString(), future2); - CompletableFuture.delayedExecutor(2, TimeUnit.SECONDS).execute(() -> { + CompletableFuture.delayedExecutor(1, TimeUnit.SECONDS).execute(() -> { future1.completeExceptionally(new CompletionException( new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); future2.completeExceptionally(new CompletionException( @@ -417,7 +419,7 @@ public boolean test(NamespaceBundle namespaceBundle) { } } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 1000) public void testNamespaceOwnershipListener() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-namespace-ownership-listener"); @@ -457,14 +459,17 @@ public boolean test(NamespaceBundle namespaceBundle) { assertEquals(unloadCount.get(), 0); }); - ServiceUnitStateChannelImpl channel = new ServiceUnitStateChannelImpl(pulsar1); - channel.start(); + @Cleanup + ServiceUnitStateChannelImpl channel3 = new ServiceUnitStateChannelImpl(pulsar1); + channel3.start(); + @Cleanup + ServiceUnitStateChannelImpl channel4 = new ServiceUnitStateChannelImpl(pulsar2); + channel4.start(); Awaitility.await().untilAsserted(() -> { assertEquals(onloadCount.get(), 2); assertEquals(unloadCount.get(), 0); }); - channel.close(); } @DataProvider(name = "isPersistentTopicSubscriptionTypeTest") @@ -484,30 +489,31 @@ public Object[][] isPersistentTopicSubscriptionTypeTest() { @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, - lookupUrl.toString(), pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); + testTransferClientReconnectionWithoutLookup(clients, topicDomain, subscriptionType, defaultTestNamespace, + admin, lookupUrl.toString(), pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); } @Test(enabled = false) - public static void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, - SubscriptionType subscriptionType, - String defaultTestNamespace, - PulsarAdmin admin, String brokerServiceUrl, - PulsarService pulsar1, PulsarService pulsar2, - ExtensibleLoadManager primaryLoadManager, - ExtensibleLoadManager secondaryLoadManager) + public static void testTransferClientReconnectionWithoutLookup( + List clients, + TopicDomain topicDomain, + SubscriptionType subscriptionType, + String defaultTestNamespace, + PulsarAdmin admin, String brokerServiceUrl, + PulsarService pulsar1, PulsarService pulsar2, + ExtensibleLoadManager primaryLoadManager, + ExtensibleLoadManager secondaryLoadManager) throws Exception { var id = String.format("test-tx-client-reconnect-%s-%s", subscriptionType, UUID.randomUUID()); var topic = String.format("%s://%s/%s", topicDomain.toString(), defaultTestNamespace, id); var topicName = TopicName.get(topic); var timeoutMs = 30_000; - var clients = new ArrayList(); var consumers = new ArrayList>(); + var lookups = new ArrayList>(); + int clientId = 0; try { - var lookups = new ArrayList(); - var pulsarClient = pulsarClient(brokerServiceUrl, 0); - clients.add(pulsarClient); + var pulsarClient = clients.get(clientId++); @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); lookups.add(spyLookupService(pulsarClient)); @@ -515,8 +521,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; for (int i = 0; i < consumerCount; i++) { - var client = pulsarClient(brokerServiceUrl, 0); - clients.add(client); + var client = clients.get(clientId++); var consumer = client.newConsumer(Schema.STRING). subscriptionName(id). subscriptionType(subscriptionType). @@ -544,8 +549,8 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic } checkOwnershipState(broker, bundle, primaryLoadManager, secondaryLoadManager, pulsar1); - var messageCountBeforeUnloading = 100; - var messageCountAfterUnloading = 100; + var messageCountBeforeUnloading = 10; + var messageCountAfterUnloading = 10; var messageCount = messageCountBeforeUnloading + messageCountAfterUnloading; var semMessagesReadyToSend = new Semaphore(0); @@ -561,6 +566,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic semMessagesReadyToSend.release(messageCountBeforeUnloading); admin.namespaces() .unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), dstBrokerUrl); + //log.info("### unloaded."); semMessagesReadyToSend.release(messageCountAfterUnloading); } catch (InterruptedException | PulsarAdminException e) { fail(); @@ -579,6 +585,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic pendingMessages.add(message); } producer.send(message); + //log.info("### producer sent: {}", message); } } catch (PulsarClientException | InterruptedException e) { fail(); @@ -594,10 +601,11 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic } while (!producerFuture.isDone() || !pendingMessages.isEmpty()) { try { - var message = consumer.receive(1500, TimeUnit.MILLISECONDS); + var message = consumer.receive(200, TimeUnit.MILLISECONDS); if (message != null) { consumer.acknowledge(message); pendingMessages.remove(message.getValue()); + //log.info("### consumer received: {}", message.getValue()); } } catch (PulsarClientException e) { // Retry read @@ -620,15 +628,17 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic assertTrue(producer.isConnected()); assertTrue(consumers.stream().allMatch(Consumer::isConnected)); - for (LookupService lookupService : lookups) { - verify(lookupService, never()).getBroker(topicName); + for (var lookupService : lookups) { + verify(lookupService.getRight(), never()).getBroker(topicName); } } finally { for (var consumer: consumers) { consumer.close(); } - for (var client: clients) { - client.close(); + + clientId = 0; + for (var lookup : lookups) { + resetLookupService(clients.get(clientId++), lookup.getLeft()); } } } @@ -636,12 +646,13 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, - lookupUrl.toString(), pulsar1); + testUnloadClientReconnectionWithLookup(clients, topicDomain, subscriptionType, defaultTestNamespace, + admin, lookupUrl.toString(), pulsar1); } @Test(enabled = false) - public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, + public static void testUnloadClientReconnectionWithLookup(List clients, + TopicDomain topicDomain, SubscriptionType subscriptionType, String defaultTestNamespace, PulsarAdmin admin, @@ -653,9 +664,10 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai var topicName = TopicName.get(topic); var consumers = new ArrayList>(); + Pair lookup = null; + PulsarClient pulsarClient = null; try { - @Cleanup - var pulsarClient = pulsarClient(brokerServiceUrl, 0); + pulsarClient = clients.get(0); var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; @@ -666,7 +678,7 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai Awaitility.await() .until(() -> producer.isConnected() && consumers.stream().allMatch(Consumer::isConnected)); - var lookup = spyLookupService(pulsarClient); + lookup = spyLookupService(pulsarClient); final CountDownLatch cdl = new CountDownLatch(3); @@ -687,7 +699,7 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai boolean messageSent = false; while (true) { var recvFutures = consumers.stream(). - map(consumer -> consumer.receiveAsync().orTimeout(1000, TimeUnit.MILLISECONDS)). + map(consumer -> consumer.receiveAsync().orTimeout(200, TimeUnit.MILLISECONDS)). collect(Collectors.toList()); if (!messageSent) { @@ -712,11 +724,12 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai assertTrue(producer.isConnected()); assertTrue(consumers.stream().allMatch(Consumer::isConnected)); assertTrue(unloadNamespaceBundle.isDone()); - verify(lookup, times(1 + consumerCount)).getBroker(topicName); + verify(lookup.getRight(), times(1 + consumerCount)).getBroker(topicName); } finally { for (var consumer : consumers) { consumer.close(); } + resetLookupService(pulsarClient, lookup.getLeft()); } } @@ -727,11 +740,13 @@ public Object[][] isPersistentTopicTest() { @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { - testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, lookupUrl.toString(), pulsar1, pulsar2); + testOptimizeUnloadDisable(clients, topicDomain, defaultTestNamespace, admin, lookupUrl.toString(), pulsar1, + pulsar2); } @Test(enabled = false) - public static void testOptimizeUnloadDisable(TopicDomain topicDomain, + public static void testOptimizeUnloadDisable(List clients, + TopicDomain topicDomain, String defaultTestNamespace, PulsarAdmin admin, String brokerServiceUrl, @@ -744,9 +759,8 @@ public static void testOptimizeUnloadDisable(TopicDomain topicDomain, pulsar1.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); pulsar2.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); - @Cleanup - var pulsarClient = pulsarClient(brokerServiceUrl, 0); - + var pulsarClient = clients.get(0); + Pair lookup = null; @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); @@ -755,64 +769,77 @@ public static void testOptimizeUnloadDisable(TopicDomain topicDomain, Awaitility.await().until(() -> producer.isConnected() && consumer.isConnected()); - var lookup = spyLookupService(pulsarClient); - - final CountDownLatch cdl = new CountDownLatch(3); + try { + lookup = spyLookupService(pulsarClient); - NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); - var srcBrokerServiceUrl = admin.lookups().lookupTopic(topic); - var dstBroker = srcBrokerServiceUrl.equals(pulsar1.getBrokerServiceUrl()) ? pulsar2 : pulsar1; + final CountDownLatch cdl = new CountDownLatch(3); - CompletableFuture unloadNamespaceBundle = CompletableFuture.runAsync(() -> { - try { - cdl.await(); - admin.namespaces().unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), - dstBroker.getBrokerId()); - } catch (InterruptedException | PulsarAdminException e) { - fail(); - } - }); + NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); + var srcBrokerServiceUrl = admin.lookups().lookupTopic(topic); + var dstBroker = srcBrokerServiceUrl.equals(pulsar1.getBrokerServiceUrl()) ? pulsar2 : pulsar1; - MutableInt sendCount = new MutableInt(); - Awaitility.await().atMost(20, TimeUnit.SECONDS).ignoreExceptions().until(() -> { - var message = String.format("message-%d", sendCount.getValue()); - - AtomicBoolean messageSent = new AtomicBoolean(false); - while (true) { - var recvFuture = consumer.receiveAsync().orTimeout(1000, TimeUnit.MILLISECONDS); - if (!messageSent.get()) { - producer.sendAsync(message).thenAccept(messageId -> { - if (messageId != null) { - messageSent.set(true); - } - }).get(1000, TimeUnit.MILLISECONDS); + CompletableFuture unloadNamespaceBundle = CompletableFuture.runAsync(() -> { + try { + cdl.await(); + admin.namespaces().unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), + dstBroker.getBrokerId()); + } catch (InterruptedException | PulsarAdminException e) { + fail(); } + }); - if (topicDomain == TopicDomain.non_persistent) { - // No need to wait for message receipt, we're only trying to stress the consumer lookup pathway. - break; - } - var msg = recvFuture.get(); - if (Objects.equals(msg.getValue(), message)) { - break; + MutableInt sendCount = new MutableInt(); + Awaitility.await().atMost(20, TimeUnit.SECONDS).ignoreExceptions().until(() -> { + var message = String.format("message-%d", sendCount.getValue()); + + AtomicBoolean messageSent = new AtomicBoolean(false); + while (true) { + var recvFuture = consumer.receiveAsync().orTimeout(200, TimeUnit.MILLISECONDS); + if (!messageSent.get()) { + producer.sendAsync(message).thenAccept(messageId -> { + if (messageId != null) { + messageSent.set(true); + } + }).get(200, TimeUnit.MILLISECONDS); + } + + if (topicDomain == TopicDomain.non_persistent) { + // No need to wait for message receipt, we're only trying to stress the consumer lookup pathway. + break; + } + var msg = recvFuture.get(); + if (Objects.equals(msg.getValue(), message)) { + break; + } } - } - cdl.countDown(); - return sendCount.incrementAndGet() == 10; - }); + cdl.countDown(); + return sendCount.incrementAndGet() == 10; + }); - assertTrue(producer.isConnected()); - assertTrue(consumer.isConnected()); - assertTrue(unloadNamespaceBundle.isDone()); - verify(lookup, times(2)).getBroker(topicName); + Pair finalLookup = lookup; + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertTrue(producer.isConnected()); + assertTrue(consumer.isConnected()); + assertTrue(unloadNamespaceBundle.isDone()); + verify(finalLookup.getRight(), times(2)).getBroker(topicName); + }); + } finally { + resetLookupService(pulsarClient, lookup.getLeft()); + } } - protected static LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { + protected static Pair spyLookupService(PulsarClient client) throws IllegalAccessException { LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); var lookup = spy(svc); FieldUtils.writeDeclaredField(client, "lookup", lookup, true); - return lookup; + return Pair.of(svc, lookup); + } + + protected static void resetLookupService(PulsarClient client, LookupService lookup) throws IllegalAccessException { + FieldUtils.writeDeclaredField(client, "lookup", lookup, true); } protected static void checkOwnershipState(String broker, NamespaceBundle bundle, @@ -1043,52 +1070,117 @@ public CompletableFuture> filterAsync(Map topicAndBundle = - getBundleIsNotOwnByChangeEventTopic("testDeployAndRollbackLoadManager"); - TopicName topicName = topicAndBundle.getLeft(); - NamespaceBundle bundle = topicAndBundle.getRight(); - String topic = topicName.toString(); - - String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult1, pulsar3.getBrokerServiceUrl()); - - String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); - String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult1, lookupResult2); - assertEquals(lookupResult1, lookupResult3); - - LookupOptions options = LookupOptions.builder() - .authoritative(false) - .requestHttps(false) - .readOnly(false) - .loadTopicsInBundle(false).build(); - Optional webServiceUrl1 = + // Test rollback to modular load manager. + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + // start pulsar3 with old load manager + @Cleanup + var pulsar3 = additionalPulsarTestContext.getPulsarService(); + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("testDeployAndRollbackLoadManager"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + String topic = topicName.toString(); + + String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, pulsar3.getBrokerServiceUrl()); + + String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, lookupResult2); + assertEquals(lookupResult1, lookupResult3); + + LookupOptions options = LookupOptions.builder() + .authoritative(false) + .requestHttps(false) + .readOnly(false) + .loadTopicsInBundle(false).build(); + Optional webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); + + Optional webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + Optional webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + } + + // Test deploy new broker with new load manager + ServiceConfiguration conf = getDefaultConf(); + conf.setAllowAutoTopicCreation(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { + @Cleanup + var pulsar4 = additionPulsarTestContext.getPulsarService(); + + Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), + pulsar2.getBrokerServiceUrl(), + pulsar4.getBrokerServiceUrl()); + String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); + assertTrue(availableCandidates.contains(lookupResult4)); + + String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult4, lookupResult5); + assertEquals(lookupResult4, lookupResult6); + assertEquals(lookupResult4, lookupResult7); + + Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), + pulsar2.getWebServiceAddress(), + pulsar4.getWebServiceAddress()); + + webServiceUrl1 = pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl1.isPresent()); - assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); - Optional webServiceUrl2 = + webServiceUrl2 = pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl2.isPresent()); assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - Optional webServiceUrl3 = + // The pulsar3 will redirect to pulsar4 + webServiceUrl3 = pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl3.isPresent()); - assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + // It will redirect to pulsar4 + assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); + + var webServiceUrl4 = + pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl4.isPresent()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); - List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); for (PulsarService pulsarService : pulsarServices) { // Test lookup heartbeat namespace's topic for (PulsarService pulsar : pulsarServices) { @@ -1101,111 +1193,269 @@ public void testDeployAndRollbackLoadManager() throws Exception { pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); + } + } + } - // Test deploy new broker with new load manager - ServiceConfiguration conf = getDefaultConf(); - conf.setAllowAutoTopicCreation(true); - conf.setForceDeleteNamespaceAllowed(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { - var pulsar4 = additionPulsarTestContext.getPulsarService(); - - Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), - pulsar2.getBrokerServiceUrl(), - pulsar4.getBrokerServiceUrl()); - String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); - assertTrue(availableCandidates.contains(lookupResult4)); - - String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); - String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); - String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult4, lookupResult5); - assertEquals(lookupResult4, lookupResult6); - assertEquals(lookupResult4, lookupResult7); - - Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), - pulsar2.getWebServiceAddress(), - pulsar4.getWebServiceAddress()); - - webServiceUrl1 = - pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl1.isPresent()); - assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); - - webServiceUrl2 = - pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl2.isPresent()); - assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - - // The pulsar3 will redirect to pulsar4 - webServiceUrl3 = - pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl3.isPresent()); - // It will redirect to pulsar4 - assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); - - var webServiceUrl4 = - pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl4.isPresent()); - assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); - - pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); - for (PulsarService pulsarService : pulsarServices) { - // Test lookup heartbeat namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupHeartbeatOwner(pulsarService, - pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); - } - // Test lookup SLA namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupSLANamespaceOwner(pulsarService, - pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); - } + @Test(priority = 200) + public void testLoadBalancerServiceUnitTableViewSyncer() throws Exception { + + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("testLoadBalancerServiceUnitTableViewSyncer"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + String topic = topicName.toString(); + + String lookupResultBefore1 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResultBefore2 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResultBefore1, lookupResultBefore2); + + LookupOptions options = LookupOptions.builder() + .authoritative(false) + .requestHttps(false) + .readOnly(false) + .loadTopicsInBundle(false).build(); + Optional webServiceUrlBefore1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrlBefore1.isPresent()); + + Optional webServiceUrlBefore2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrlBefore2.isPresent()); + assertEquals(webServiceUrlBefore2.get().toString(), webServiceUrlBefore1.get().toString()); + + + String syncerTyp = serviceUnitStateTableViewClassName.equals(ServiceUnitStateTableViewImpl.class.getName()) ? + "SystemTopicToMetadataStoreSyncer" : "MetadataStoreToSystemTopicSyncer"; + pulsar.getAdminClient().brokers() + .updateDynamicConfiguration("loadBalancerServiceUnitTableViewSyncer", syncerTyp); + makeSecondaryAsLeader(); + makePrimaryAsLeader(); + Awaitility.waitAtMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(primaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + Awaitility.waitAtMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(secondaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getCanonicalName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + defaultConf.setLoadManagerServiceUnitStateTableViewClassName(ServiceUnitStateTableViewImpl.class.getName()); + try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + // start pulsar3 with ServiceUnitStateTableViewImpl + @Cleanup + var pulsar3 = additionalPulsarTestContext.getPulsarService(); + + String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, lookupResult2); + assertEquals(lookupResult1, lookupResult3); + assertEquals(lookupResult1, lookupResultBefore1); + + Optional webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + + Optional webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + Optional webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + + assertEquals(webServiceUrl3.get().toString(), webServiceUrlBefore1.get().toString()); + + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + } + + // Start broker4 with ServiceUnitStateMetadataStoreTableViewImpl + ServiceConfiguration conf = getDefaultConf(); + conf.setAllowAutoTopicCreation(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getCanonicalName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName( + ServiceUnitStateMetadataStoreTableViewImpl.class.getName()); + try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { + @Cleanup + var pulsar4 = additionPulsarTestContext.getPulsarService(); + + Set availableCandidates = Sets.newHashSet( + pulsar1.getBrokerServiceUrl(), + pulsar2.getBrokerServiceUrl(), + pulsar3.getBrokerServiceUrl(), + pulsar4.getBrokerServiceUrl()); + String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); + assertTrue(availableCandidates.contains(lookupResult4)); + + String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult4, lookupResult5); + assertEquals(lookupResult4, lookupResult6); + assertEquals(lookupResult4, lookupResult7); + assertEquals(lookupResult4, lookupResultBefore1); + + + Pair topicAndBundle2 = + getBundleIsNotOwnByChangeEventTopic("testLoadBalancerServiceUnitTableViewSyncer2"); + String topic2 = topicAndBundle2.getLeft().toString(); + + String lookupResult8 = pulsar1.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult9 = pulsar2.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult10 = pulsar3.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult11 = pulsar4.getAdminClient().lookups().lookupTopic(topic2); + assertEquals(lookupResult9, lookupResult8); + assertEquals(lookupResult10, lookupResult8); + assertEquals(lookupResult11, lookupResult8); + + Set availableWebUrlCandidates = Sets.newHashSet( + pulsar1.getWebServiceAddress(), + pulsar2.getWebServiceAddress(), + pulsar3.getWebServiceAddress(), + pulsar4.getWebServiceAddress()); + + webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); + + webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); + + var webServiceUrl4 = + pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl4.isPresent()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrlBefore1.get().toString()); + + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } - // Check if the broker is available - var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); - var loadManager4 = spy((ExtensibleLoadManagerImpl) - FieldUtils.readField(wrapper, "loadManager", true)); - loadManager4.getBrokerRegistry().unregister(); - - NamespaceName slaMonitorNamespace = - getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); - String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); - String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); - assertNotNull(result); - log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); - assertNotEquals(result, pulsar4.getBrokerServiceUrl()); - - Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); - producer.send("t1"); - - // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); - - result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); - assertNotNull(result); - log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); - assertEquals(result, pulsar4.getBrokerServiceUrl()); - - producer.send("t2"); - Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); - producer1.send("t3"); - - producer.close(); - producer1.close(); - @Cleanup - Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) - .topic(slaMonitorTopic) - .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .subscriptionName("test") - .subscribe(); - // receive message t1 t2 t3 - assertEquals(consumer.receive().getValue(), "t1"); - assertEquals(consumer.receive().getValue(), "t2"); - assertEquals(consumer.receive().getValue(), "t3"); } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); } + } + + pulsar.getAdminClient().brokers() + .deleteDynamicConfiguration("loadBalancerServiceUnitTableViewSyncer"); + makeSecondaryAsLeader(); + Awaitility.waitAtMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(primaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + Awaitility.waitAtMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(secondaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); } private void assertLookupHeartbeatOwner(PulsarService pulsar, @@ -1272,12 +1522,13 @@ private void makeSecondaryAsLeader() throws Exception { }); } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 2100) public void testRoleChangeIdempotency() throws Exception { makePrimaryAsLeader(); - var topBundlesLoadDataStorePrimary = primaryLoadManager.getTopBundlesLoadDataStore(); + var topBundlesLoadDataStorePrimary = + (TableViewLoadDataStoreImpl) primaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStorePrimarySpy = spy(topBundlesLoadDataStorePrimary); AtomicInteger countPri = new AtomicInteger(3); AtomicInteger countPri2 = new AtomicInteger(3); @@ -1286,19 +1537,18 @@ public void testRoleChangeIdempotency() throws Exception { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStorePrimarySpy).startTableView(); doAnswer(invocationOnMock -> { if (countPri2.decrementAndGet() > 0) { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStorePrimarySpy).closeTableView(); - var topBundlesLoadDataStoreSecondary = secondaryLoadManager.getTopBundlesLoadDataStore(); + var topBundlesLoadDataStoreSecondary = + (TableViewLoadDataStoreImpl) secondaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStoreSecondarySpy = spy(topBundlesLoadDataStoreSecondary); AtomicInteger countSec = new AtomicInteger(3); AtomicInteger countSec2 = new AtomicInteger(3); @@ -1306,17 +1556,14 @@ public void testRoleChangeIdempotency() throws Exception { if (countSec.decrementAndGet() > 0) { throw new RuntimeException(); } - // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStoreSecondarySpy).startTableView(); doAnswer(invocationOnMock -> { if (countSec2.decrementAndGet() > 0) { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStoreSecondarySpy).closeTableView(); try { @@ -1325,8 +1572,6 @@ public void testRoleChangeIdempotency() throws Exception { FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); - - primaryLoadManager.playLeader(); secondaryLoadManager.playFollower(); verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); @@ -1334,6 +1579,13 @@ public void testRoleChangeIdempotency() throws Exception { verify(topBundlesLoadDataStoreSecondarySpy, times(0)).startTableView(); verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + } finally { + FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStorePrimary, true); + FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStoreSecondary, true); + } + primaryLoadManager.playFollower(); secondaryLoadManager.playFollower(); @@ -1350,14 +1602,9 @@ public void testRoleChangeIdempotency() throws Exception { assertEquals(ExtensibleLoadManagerImpl.Role.Follower, secondaryLoadManager.getRole()); - } finally { - FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", - topBundlesLoadDataStorePrimary, true); - FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", - topBundlesLoadDataStoreSecondary, true); - } + } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 2000) public void testRoleChange() throws Exception { makePrimaryAsLeader(); @@ -1375,15 +1622,18 @@ public void testRoleChange() throws Exception { topBundlesExpected.getTopBundlesLoadData().clear(); topBundlesExpected.getTopBundlesLoadData().add(new TopBundlesLoadData.BundleLoadData(bundle, new NamespaceBundleStats())); - follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); - follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); - Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { assertNotNull(FieldUtils.readDeclaredField(leader.getTopBundlesLoadDataStore(), "tableView", true)); assertNull(FieldUtils.readDeclaredField(follower.getTopBundlesLoadDataStore(), "tableView", true)); + for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + if (serviceUnitStateTableViewClassName + .equals(ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName()) + && internalTopic.equals(TOPIC)) { + continue; + } assertTrue(leader.pulsar.getBrokerService().getTopicReference(internalTopic) .isPresent()); assertTrue(follower.pulsar.getBrokerService().getTopicReference(internalTopic) @@ -1394,22 +1644,9 @@ public void testRoleChange() throws Exception { assertFalse(follower.pulsar.getNamespaceService() .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); } - - var actualBrokerLoadLeader = leader.getBrokerLoadDataStore().get(key); - if (actualBrokerLoadLeader.isPresent()) { - assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); - } - - var actualTopBundlesLeader = leader.getTopBundlesLoadDataStore().get(bundle); - if (actualTopBundlesLeader.isPresent()) { - assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); - } - - var actualBrokerLoadFollower = follower.getBrokerLoadDataStore().get(key); - if (actualBrokerLoadFollower.isPresent()) { - assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); - } }); + follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected).get(3, TimeUnit.SECONDS); + follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected).get(3, TimeUnit.SECONDS); makeSecondaryAsLeader(); @@ -1419,14 +1656,16 @@ public void testRoleChange() throws Exception { brokerLoadExpected.update(usage, 1, 0, 0, 0, 0, 0, conf); topBundlesExpected.getTopBundlesLoadData().get(0).stats().msgRateIn = 1; - follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); - follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); - Awaitility.await().atMost(30, TimeUnit.SECONDS).ignoreExceptions().untilAsserted(() -> { assertNotNull(FieldUtils.readDeclaredField(leader2.getTopBundlesLoadDataStore(), "tableView", true)); assertNull(FieldUtils.readDeclaredField(follower2.getTopBundlesLoadDataStore(), "tableView", true)); for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + if (serviceUnitStateTableViewClassName + .equals(ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName()) + && internalTopic.equals(TOPIC)) { + continue; + } assertTrue(leader2.pulsar.getBrokerService().getTopicReference(internalTopic) .isPresent()); assertTrue(follower2.pulsar.getBrokerService().getTopicReference(internalTopic) @@ -1437,17 +1676,10 @@ public void testRoleChange() throws Exception { assertFalse(follower2.pulsar.getNamespaceService() .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); } - - - var actualBrokerLoadLeader = leader2.getBrokerLoadDataStore().get(key); - assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); - - var actualTopBundlesLeader = leader2.getTopBundlesLoadDataStore().get(bundle); - assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); - - var actualBrokerLoadFollower = follower2.getBrokerLoadDataStore().get(key); - assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); }); + + follower2.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected).get(3, TimeUnit.SECONDS); + follower2.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected).get(3, TimeUnit.SECONDS); } @Test @@ -1647,7 +1879,7 @@ SplitDecision.Reason.Unknown, new AtomicLong(6)) assertEquals(actual, expected); } - @Test + @Test(priority = 100) public void testDisableBroker() throws Exception { // Test rollback to modular load manager. ServiceConfiguration defaultConf = getDefaultConf(); @@ -1658,7 +1890,9 @@ public void testDisableBroker() throws Exception { defaultConf.setLoadBalancerSheddingEnabled(false); defaultConf.setLoadBalancerDebugModeEnabled(true); defaultConf.setTopicLevelPoliciesEnabled(false); + defaultConf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + @Cleanup var pulsar3 = additionalPulsarTestContext.getPulsarService(); ExtensibleLoadManagerImpl ternaryLoadManager = spy((ExtensibleLoadManagerImpl) FieldUtils.readField(pulsar3.getLoadManager().get(), "loadManager", true)); @@ -1767,15 +2001,13 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio .getFullBundle(slaMonitorNamespacePulsar2); - Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); assertTrue(ownedServiceUnitsByPulsar1.contains(slaBundle1)); - Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); @@ -1811,8 +2043,7 @@ private void assertOwnedServiceUnits( ExtensibleLoadManagerImpl extensibleLoadManager, NamespaceBundle bundle) throws PulsarAdminException { Awaitility.await().untilAsserted(() -> { - Set ownedBundles = extensibleLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedBundles = extensibleLoadManager.getOwnedServiceUnits(); assertTrue(ownedBundles.contains(bundle)); }); Map ownedNamespaces = @@ -1828,8 +2059,7 @@ private void assertOwnedServiceUnits( public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() throws Exception { ExtensibleLoadManagerImpl loadManager = new ExtensibleLoadManagerImpl(); - Set ownedServiceUnits = loadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnits = loadManager.getOwnedServiceUnits(); assertNotNull(ownedServiceUnits); assertTrue(ownedServiceUnits.isEmpty()); } @@ -1858,6 +2088,11 @@ public void testHealthcheck() throws PulsarAdminException { @Test(timeOut = 30 * 1000) public void compactionScheduleTest() { + if (serviceUnitStateTableViewClassName.equals( + ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName())) { + // no topic compaction happens + return; + } Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(30, TimeUnit.SECONDS) @@ -1866,37 +2101,11 @@ public void compactionScheduleTest() { primaryLoadManager.monitor(); secondaryLoadManager.monitor(); var threshold = admin.topicPolicies() - .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, false); + .getCompactionThreshold(TOPIC, false); AssertJUnit.assertEquals(5 * 1024 * 1024, threshold == null ? 0 : threshold.longValue()); }); } - @Test(timeOut = 10 * 1000) - public void unloadTimeoutCheckTest() - throws Exception { - Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("unload-timeout"); - String topic = topicAndBundle.getLeft().toString(); - var bundle = topicAndBundle.getRight().toString(); - var releasing = new ServiceUnitStateData(Releasing, pulsar2.getBrokerId(), pulsar1.getBrokerId(), 1); - overrideTableView(channel1, bundle, releasing); - var topicFuture = pulsar1.getBrokerService().getOrCreateTopic(topic); - - - try { - topicFuture.get(1, TimeUnit.SECONDS); - } catch (Exception e) { - log.info("getOrCreateTopic failed", e); - if (!(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException && e.getMessage() - .contains("Please redo the lookup"))) { - fail(); - } - } - - pulsar1.getBrokerService() - .unloadServiceUnit(topicAndBundle.getRight(), true, true, 5, - TimeUnit.SECONDS).get(2, TimeUnit.SECONDS); - } - private static abstract class MockBrokerFilter implements BrokerFilter { @Override @@ -1905,12 +2114,4 @@ public String name() { } } - - protected static PulsarClient pulsarClient(String url, int intervalInSecs) throws PulsarClientException { - return - PulsarClient.builder() - .serviceUrl(url) - .statsInterval(intervalInSecs, TimeUnit.SECONDS).build(); - } - } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java index bec7d4d78fe7e..b9c945fe81571 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java @@ -25,6 +25,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicDomain; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; /** @@ -36,8 +37,10 @@ public class ExtensibleLoadManagerImplWithAdvertisedListenersTest extends ExtensibleLoadManagerImplBaseTest { public String brokerServiceUrl; - public ExtensibleLoadManagerImplWithAdvertisedListenersTest() { - super("public/test"); + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplWithAdvertisedListenersTest(String serviceUnitStateTableViewClassName) { + super("public/test", serviceUnitStateTableViewClassName); } @Override @@ -69,7 +72,9 @@ public Object[][] isPersistentTopicSubscriptionTypeTest() { @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - ExtensibleLoadManagerImplTest.testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, + ExtensibleLoadManagerImplTest.testTransferClientReconnectionWithoutLookup( + clients, + topicDomain, subscriptionType, defaultTestNamespace, admin, brokerServiceUrl, pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); @@ -78,7 +83,9 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - ExtensibleLoadManagerImplTest.testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, + ExtensibleLoadManagerImplTest.testUnloadClientReconnectionWithLookup( + clients, + topicDomain, subscriptionType, defaultTestNamespace, admin, brokerServiceUrl, pulsar1); @@ -91,7 +98,9 @@ public Object[][] isPersistentTopicTest() { @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { - ExtensibleLoadManagerImplTest.testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, + ExtensibleLoadManagerImplTest.testOptimizeUnloadDisable( + clients, + topicDomain, defaultTestNamespace, admin, brokerServiceUrl, pulsar1, pulsar2); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java index ed99b502b7e29..1d3f02f4e717d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java @@ -21,13 +21,15 @@ import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.ServiceConfiguration; import org.awaitility.Awaitility; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker") public class ExtensibleLoadManagerImplWithTransactionCoordinatorTest extends ExtensibleLoadManagerImplBaseTest { - public ExtensibleLoadManagerImplWithTransactionCoordinatorTest() { - super("public/test-elb-with-tx"); + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplWithTransactionCoordinatorTest(String serviceUnitStateTableViewClassName) { + super("public/test-elb-with-tx", serviceUnitStateTableViewClassName); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index e569f0d32d573..92cdf61f44269 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -30,6 +30,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MAX_CLEAN_UP_DELAY_TIME_IN_SECS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.metadata.api.extended.SessionEvent.ConnectionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.Reconnected; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -37,23 +38,22 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.expectThrows; -import static org.testng.Assert.fail; -import static org.testng.AssertJUnit.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; +import static org.testng.Assert.fail; +import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNull; import java.lang.reflect.Field; @@ -70,13 +70,14 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import lombok.Cleanup; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistryImpl; @@ -86,12 +87,14 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.testcontext.PulsarTestContext; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.TypedMessageBuilder; -import org.apache.pulsar.client.impl.TableViewImpl; +import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.apache.pulsar.metadata.api.extended.SessionEvent; @@ -99,6 +102,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker") @@ -109,6 +114,8 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private PulsarService pulsar2; private ServiceUnitStateChannel channel1; private ServiceUnitStateChannel channel2; + private String namespaceName; + private String namespaceName2; private String brokerId1; private String brokerId2; private String brokerId3; @@ -131,19 +138,40 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private ExtensibleLoadManagerImpl loadManager; - @BeforeClass - @Override - protected void setup() throws Exception { + private final String serviceUnitStateTableViewClassName; + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {ServiceUnitStateTableViewImpl.class.getName()}, + {ServiceUnitStateMetadataStoreTableViewImpl.class.getName()} + }; + } + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ServiceUnitStateChannelTest(String serviceUnitStateTableViewClassName) { + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; + } + + private void updateConfig(ServiceConfiguration conf) { conf.setAllowAutoTopicCreation(true); conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); conf.setLoadBalancerDebugModeEnabled(true); conf.setBrokerServiceCompactionMonitorIntervalInSeconds(10); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + } + + @BeforeClass + @Override + protected void setup() throws Exception { + updateConfig(conf); super.internalSetup(conf); - admin.tenants().createTenant("pulsar", createDefaultTenantInfo()); - admin.namespaces().createNamespace("pulsar/system"); - admin.tenants().createTenant("public", createDefaultTenantInfo()); - admin.namespaces().createNamespace("public/default"); + namespaceName = "my-tenant/my-ns"; + namespaceName2 = "my-tenant/my-ns2"; + admin.tenants().createTenant("my-tenant", createDefaultTenantInfo()); + admin.namespaces().createNamespace(namespaceName); + admin.namespaces().createNamespace(namespaceName2); pulsar1 = pulsar; registry = new BrokerRegistryImpl(pulsar); @@ -151,7 +179,9 @@ protected void setup() throws Exception { doReturn(mock(LoadDataStore.class)).when(loadManagerContext).brokerLoadDataStore(); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).topBundleLoadDataStore(); loadManager = mock(ExtensibleLoadManagerImpl.class); - additionalPulsarTestContext = createAdditionalPulsarTestContext(getDefaultConf()); + var conf2 = getDefaultConf(); + updateConfig(conf2); + additionalPulsarTestContext = createAdditionalPulsarTestContext(conf2); pulsar2 = additionalPulsarTestContext.getPulsarService(); channel1 = createChannel(pulsar1); @@ -165,22 +195,23 @@ protected void setup() throws Exception { FieldUtils.readDeclaredField(channel2, "brokerId", true); brokerId3 = "broker-3"; - bundle = "public/default/0x00000000_0xffffffff"; - bundle1 = "public/default/0x00000000_0xfffffff0"; - bundle2 = "public/default/0xfffffff0_0xffffffff"; - bundle3 = "public/default3/0x00000000_0xffffffff"; + bundle = namespaceName + "/0x00000000_0xffffffff"; + bundle1 = namespaceName + "/0x00000000_0xfffffff0"; + bundle2 = namespaceName + "/0xfffffff0_0xffffffff"; + bundle3 = namespaceName2 + "/0x00000000_0xffffffff"; childBundle1Range = "0x7fffffff_0xffffffff"; childBundle2Range = "0x00000000_0x7fffffff"; - childBundle11 = "public/default/" + childBundle1Range; - childBundle12 = "public/default/" + childBundle2Range; + childBundle11 = namespaceName + "/" + childBundle1Range; + childBundle12 = namespaceName + "/" + childBundle2Range; - childBundle31 = "public/default3/" + childBundle1Range; - childBundle32 = "public/default3/" + childBundle2Range; + childBundle31 = namespaceName2 + "/" + childBundle1Range; + childBundle32 = namespaceName2 + "/" + childBundle2Range; } @BeforeMethod protected void initChannels() throws Exception { + disableChannels(); cleanTableViews(); cleanOwnershipMonitorCounters(channel1); cleanOwnershipMonitorCounters(channel2); @@ -188,6 +219,7 @@ protected void initChannels() throws Exception { cleanOpsCounters(channel2); cleanMetadataState(channel1); cleanMetadataState(channel2); + enableChannels(); } @@ -205,7 +237,7 @@ protected void cleanup() throws Exception { super.internalCleanup(); } - @Test(priority = -1) + @Test(priority = 0) public void channelOwnerTest() throws Exception { var channelOwner1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); var channelOwner2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); @@ -232,7 +264,7 @@ public void channelOwnerTest() throws Exception { } } - @Test(priority = 0) + @Test(priority = 100) public void channelValidationTest() throws ExecutionException, InterruptedException, IllegalAccessException, PulsarServerException, TimeoutException { @@ -256,7 +288,7 @@ public void channelValidationTest() ServiceUnitStateChannelImpl.ChannelState.LeaderElectionServiceStarted, true); assertNotNull(channel.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get()); - Future closeFuture = executor.submit(()->{ + Future closeFuture = executor.submit(() -> { try { channel.close(); } catch (PulsarServerException e) { @@ -289,7 +321,7 @@ private int validateChannelStart(ServiceUnitStateChannelImpl channel) try { channel.isChannelOwnerAsync().get(2, TimeUnit.SECONDS); } catch (ExecutionException e) { - if(e.getCause() instanceof IllegalStateException){ + if (e.getCause() instanceof IllegalStateException) { errorCnt++; } } @@ -316,7 +348,7 @@ private int validateChannelStart(ServiceUnitStateChannelImpl channel) } try { channel.publishUnloadEventAsync( - new Unload(brokerId1, bundle, Optional.of(brokerId2))) + new Unload(brokerId1, bundle, Optional.of(brokerId2))) .get(2, TimeUnit.SECONDS); } catch (ExecutionException e) { if (e.getCause() instanceof IllegalStateException) { @@ -382,35 +414,33 @@ public void assignmentTestWhenOneAssignmentFails() assertEquals(0, getOwnerRequests1.size()); assertEquals(0, getOwnerRequests2.size()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); - var future = spy(CompletableFuture.failedFuture(new RuntimeException())); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - - FieldUtils.writeDeclaredField(channel1, "producer", spyProducer, true); - - var owner1 = channel1.getOwnerAsync(bundle); - var owner2 = channel2.getOwnerAsync(bundle); - - assertTrue(owner1.get().isEmpty()); - assertTrue(owner2.get().isEmpty()); + var tableView = getTableView(channel1); + var spyTableView = spy(tableView); + var future = CompletableFuture.failedFuture(new RuntimeException()); + doReturn(future).when(spyTableView).put(any(), any()); - var owner3 = channel1.publishAssignEventAsync(bundle, brokerId1); - var owner4 = channel2.publishAssignEventAsync(bundle, brokerId2); - assertTrue(owner3.isCompletedExceptionally()); - assertNotNull(owner4); - String ownerAddrOpt2 = owner4.get(5, TimeUnit.SECONDS); - assertEquals(ownerAddrOpt2, brokerId2); - waitUntilNewOwner(channel1, bundle, brokerId2); - assertEquals(0, getOwnerRequests1.size()); - assertEquals(0, getOwnerRequests2.size()); + try { + setTableView(channel1, spyTableView); + + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertTrue(owner1.get().isEmpty()); + assertTrue(owner2.get().isEmpty()); + var owner3 = channel1.publishAssignEventAsync(bundle, brokerId1); + var owner4 = channel2.publishAssignEventAsync(bundle, brokerId2); + + assertTrue(owner3.isCompletedExceptionally()); + assertNotNull(owner4); + String ownerAddrOpt2 = owner4.get(5, TimeUnit.SECONDS); + assertEquals(ownerAddrOpt2, brokerId2); + waitUntilNewOwner(channel1, bundle, brokerId2); + assertEquals(0, getOwnerRequests1.size()); + assertEquals(0, getOwnerRequests2.size()); + } finally { + setTableView(channel1, tableView); + } - FieldUtils.writeDeclaredField(channel1, "producer", producer, true); } @Test(priority = 4) @@ -423,7 +453,6 @@ public void transferTest() assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); - channel1.publishAssignEventAsync(bundle, brokerId1); waitUntilNewOwner(channel1, bundle, brokerId1); waitUntilNewOwner(channel2, bundle, brokerId1); @@ -468,78 +497,78 @@ public void transferTestWhenDestBrokerFails() assertEquals(ownerAddr1, ownerAddr2); assertEquals(ownerAddr1, Optional.of(brokerId1)); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableView = getTableView(channel2); + var spyTableView = spy(tableView); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel2, "producer", spyProducer, true); - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 3 * 1000, true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 3 * 1000, true); - Unload unload = new Unload(brokerId1, bundle, Optional.of(brokerId2)); - channel1.publishUnloadEventAsync(unload); - // channel1 is broken. the ownership transfer won't be complete. - waitUntilState(channel1, bundle); - waitUntilState(channel2, bundle); - var owner1 = channel1.getOwnerAsync(bundle); - var owner2 = channel2.getOwnerAsync(bundle); - - assertTrue(owner1.isDone()); - assertEquals(brokerId2, owner1.get().get()); - assertFalse(owner2.isDone()); - - assertEquals(0, getOwnerRequests1.size()); - assertEquals(1, getOwnerRequests2.size()); - - // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); - - assertEquals(0, getOwnerRequests2.size()); - - // recovered, check the monitor update state : Assigned -> Owned - doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any(), any()); - FieldUtils.writeDeclaredField(channel2, "producer", producer, true); - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); - - ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( - List.of(brokerId1, brokerId2)); - ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( - List.of(brokerId1, brokerId2)); - - - waitUntilNewOwner(channel1, bundle, brokerId1); - waitUntilNewOwner(channel2, bundle, brokerId1); - ownerAddr1 = channel1.getOwnerAsync(bundle).get(); - ownerAddr2 = channel2.getOwnerAsync(bundle).get(); - - assertEquals(ownerAddr1, ownerAddr2); - assertEquals(ownerAddr1, Optional.of(brokerId1)); - - var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; - validateMonitorCounters(leader, - 0, - 0, - 1, - 0, - 0, - 0, - 0); + doReturn(future).when(spyTableView).put(any(), any()); + try { + setTableView(channel2, spyTableView); + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + Unload unload = new Unload(brokerId1, bundle, Optional.of(brokerId2)); + channel1.publishUnloadEventAsync(unload); + // channel2 is broken. the ownership transfer won't be complete. + waitUntilState(channel1, bundle); + waitUntilState(channel2, bundle); + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertTrue(owner1.isDone()); + assertEquals(brokerId2, owner1.get().get()); + assertFalse(owner2.isDone()); + + assertEquals(0, getOwnerRequests1.size()); + assertEquals(1, getOwnerRequests2.size()); + + // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); + + assertEquals(0, getOwnerRequests2.size()); + + // recovered, check the monitor update state : Assigned -> Owned + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); + } finally { + setTableView(channel2, tableView); + } - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + try { + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1, true); + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(brokerId1, brokerId2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(brokerId1, brokerId2)); + + + waitUntilNewOwner(channel1, bundle, brokerId1); + waitUntilNewOwner(channel2, bundle, brokerId1); + ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(brokerId1)); + + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 0, + 1, + 0, + 0, + 0, + 0); + } finally { + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + } } @@ -563,11 +592,7 @@ public void splitAndRetryTest() throws Exception { if (count.decrementAndGet() > 0) { return future; } - // Call the real method - reset(namespaceService); - doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) - .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); - return future; + return invocationOnMock.callRealMethod(); }).when(namespaceService).updateNamespaceBundles(any(), any()); doReturn(namespaceService).when(pulsar1).getNamespaceService(); doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) @@ -587,11 +612,10 @@ public void splitAndRetryTest() throws Exception { validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); // Verify the retry count - verify(((ServiceUnitStateChannelImpl) channel1), times(badVersionExceptionCount + 1)) + verify(((ServiceUnitStateChannelImpl) channel1), times(badVersionExceptionCount)) .splitServiceUnitOnceAndRetry(any(), any(), any(), any(), any(), any(), any(), any(), anyLong(), any()); - waitUntilNewOwner(channel1, childBundle11, brokerId1); waitUntilNewOwner(channel1, childBundle12, brokerId1); waitUntilNewOwner(channel2, childBundle11, brokerId1); @@ -604,12 +628,12 @@ public void splitAndRetryTest() throws Exception { // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel1, "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "stateTombstoneDelayTimeInMillis", 1, true); @@ -630,10 +654,15 @@ public void splitAndRetryTest() throws Exception { 0, 0); - cleanTableView(channel1, childBundle11); - cleanTableView(channel2, childBundle11); - cleanTableView(channel1, childBundle12); - cleanTableView(channel2, childBundle12); + try { + disableChannels(); + overrideTableView(channel1, childBundle11, null); + overrideTableView(channel2, childBundle11, null); + overrideTableView(channel1, childBundle12, null); + overrideTableView(channel2, childBundle12, null); + } finally { + enableChannels(); + } FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); @@ -649,6 +678,7 @@ public void splitAndRetryTest() throws Exception { @Test(priority = 7) public void handleMetadataSessionEventTest() throws IllegalAccessException { var ts = System.currentTimeMillis(); + ServiceUnitStateChannelImpl channel1 = (ServiceUnitStateChannelImpl) this.channel1; channel1.handleMetadataSessionEvent(SessionReestablished); var lastMetadataSessionEvent = getLastMetadataSessionEvent(channel1); var lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1); @@ -692,7 +722,7 @@ public void handleBrokerCreationEventTest() throws IllegalAccessException { String broker = "broker-1"; var future = new CompletableFuture(); cleanupJobs.put(broker, future); - channel1.handleBrokerRegistrationEvent(broker, NotificationType.Created); + ((ServiceUnitStateChannelImpl) channel1).handleBrokerRegistrationEvent(broker, NotificationType.Created); assertEquals(0, cleanupJobs.size()); assertTrue(future.isCancelled()); } @@ -705,14 +735,14 @@ public void handleBrokerDeletionEventTest() var cleanupJobs2 = getCleanupJobs(channel2); var leaderCleanupJobsTmp = spy(cleanupJobs1); var followerCleanupJobsTmp = spy(cleanupJobs2); - var leaderChannel = channel1; - var followerChannel = channel2; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; + ServiceUnitStateChannelImpl followerChannel = (ServiceUnitStateChannelImpl) channel2; String leader = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader, leader2); if (leader.equals(brokerId2)) { - leaderChannel = channel2; - followerChannel = channel1; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; + followerChannel = (ServiceUnitStateChannelImpl) channel1; var tmp = followerCleanupJobsTmp; followerCleanupJobsTmp = leaderCleanupJobsTmp; leaderCleanupJobsTmp = tmp; @@ -754,8 +784,10 @@ public void handleBrokerDeletionEventTest() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); - leaderChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); - followerChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); + leaderChannel.handleBrokerRegistrationEvent(brokerId2, + NotificationType.Deleted); + followerChannel.handleBrokerRegistrationEvent(brokerId2, + NotificationType.Deleted); waitUntilNewOwner(channel1, bundle1, brokerId2); waitUntilNewOwner(channel2, bundle1, brokerId2); @@ -912,7 +944,7 @@ public void handleBrokerDeletionEventTest() true); } - @Test(priority = 10) + @Test(priority = 2000) public void conflictAndCompactionTest() throws Exception { String bundle = String.format("%s/%s", "public/default", "0x0000000a_0xffffffff"); var owner1 = channel1.getOwnerAsync(bundle); @@ -941,16 +973,21 @@ public void conflictAndCompactionTest() throws Exception { assertNull(ex); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(bundle).get()); assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(bundle).get()); + if (serviceUnitStateTableViewClassName.equals( + ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName())) { + // no compaction + return; + } - var compactor = spy (pulsar1.getStrategicCompactor()); + var compactor = spy(pulsar1.getStrategicCompactor()); Field strategicCompactorField = FieldUtils.getDeclaredField(PulsarService.class, "strategicCompactor", true); FieldUtils.writeField(strategicCompactorField, pulsar1, compactor, true); FieldUtils.writeField(strategicCompactorField, pulsar2, compactor, true); var threshold = admin.topicPolicies() - .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC); + .getCompactionThreshold(TOPIC); admin.topicPolicies() - .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, 0); + .setCompactionThreshold(TOPIC, 0); try { Awaitility.await() @@ -959,7 +996,7 @@ public void conflictAndCompactionTest() throws Exception { .untilAsserted(() -> { channel1.publishAssignEventAsync(bundle, brokerId1); verify(compactor, times(1)) - .compact(eq(ServiceUnitStateChannelImpl.TOPIC), any()); + .compact(eq(TOPIC), any()); }); @@ -976,7 +1013,7 @@ public void conflictAndCompactionTest() throws Exception { "inFlightStateWaitingTimeInMillis", 30 * 1000, true); if (threshold != null) { admin.topicPolicies() - .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, threshold); + .setCompactionThreshold(TOPIC, threshold); } } @@ -985,36 +1022,40 @@ public void conflictAndCompactionTest() throws Exception { @Test(priority = 11) public void ownerLookupCountTests() throws IllegalAccessException { + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, "b1", 1)); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, "b1", 1)); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1", 1)); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1", 1)); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, null); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, null); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - validateOwnerLookUpCounters(channel1, 2, 3, 2, 1, 1, 2, 3); + validateOwnerLookUpCounters(channel1, 2, 3, 2, 1, 1, 2, 3); + } finally { + enableChannels(); + } } @@ -1062,12 +1103,12 @@ public void unloadTest() // test monitor if Free -> Init FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel1, "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "stateTombstoneDelayTimeInMillis", 1, true); @@ -1114,16 +1155,11 @@ public void assignTestWhenDestBrokerProducerFails() assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); assertEquals(Optional.empty(), channel2.getOwnerAsync(bundle).get()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableview = getTableView(channel1); + var tableviewSpy = spy(tableview); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel2, "producer", spyProducer, true); + doReturn(future).when(tableviewSpy).put(any(), any()); + setTableView(channel2, tableviewSpy); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); FieldUtils.writeDeclaredField(channel2, @@ -1145,11 +1181,11 @@ public void assignTestWhenDestBrokerProducerFails() .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); // recovered, check the monitor update state : Assigned -> Owned - FieldUtils.writeDeclaredField(channel2, "producer", producer, true); + setTableView(channel2, tableview); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( List.of(brokerId1, brokerId2)); @@ -1183,7 +1219,7 @@ public void assignTestWhenDestBrokerProducerFails() } @Test(priority = 14) - public void splitTestWhenProducerFails() + public void splitTestWhenTableViewPutFails() throws ExecutionException, InterruptedException, IllegalAccessException { @@ -1202,16 +1238,11 @@ public void splitTestWhenProducerFails() assertEquals(brokerId1, channel1.getOwnerAsync(bundle).get().get()); assertEquals(brokerId1, channel2.getOwnerAsync(bundle).get().get()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableview = getTableView(channel1); + var tableviewSpy = spy(tableview); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel1, "producer", spyProducer, true); + doReturn(future).when(tableviewSpy).put(any(), any()); + setTableView(channel1, tableviewSpy); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); FieldUtils.writeDeclaredField(channel2, @@ -1230,11 +1261,11 @@ public void splitTestWhenProducerFails() // recovered, check the monitor update state : Splitting -> Owned - FieldUtils.writeDeclaredField(channel1, "producer", producer, true); + setTableView(channel1, tableview); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; @@ -1261,6 +1292,7 @@ public void splitTestWhenProducerFails() @Test(priority = 15) public void testIsOwner() throws IllegalAccessException { + var owner1 = channel1.isOwner(bundle); var owner2 = channel2.isOwner(bundle); @@ -1298,91 +1330,102 @@ public void testIsOwner() throws IllegalAccessException { assertTrue(owner1); assertFalse(owner2); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); - assertTrue(channel1.isOwner(bundle)); + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); + assertTrue(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); - assertTrue(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); + assertTrue(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, null); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, null); + assertFalse(channel1.isOwner(bundle)); + } finally { + enableChannels(); + } } - @Test(priority = 15) + @Test(priority = 16) public void testGetOwnerAsync() throws Exception { + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); + var owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertFalse(owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertFalse(owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, null); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle1, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle1); + //assertTrue(owner.isDone()); + assertTrue(owner.isCompletedExceptionally()); + } finally { + enableChannels(); + } - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); - var owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId1, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(!owner.isDone()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(!owner.isDone()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId1, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertTrue(owner.isCompletedExceptionally()); - - overrideTableView(channel1, bundle, null); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); } - @Test(priority = 16) + @Test(priority = 17) public void splitAndRetryFailureTest() throws Exception { channel1.publishAssignEventAsync(bundle3, brokerId1); waitUntilNewOwner(channel1, bundle3, brokerId1); @@ -1395,6 +1438,7 @@ public void splitAndRetryFailureTest() throws Exception { NamespaceService namespaceService = pulsar1.getNamespaceService(); CompletableFuture future = new CompletableFuture<>(); + int badVersionExceptionCount = 10; AtomicInteger count = new AtomicInteger(badVersionExceptionCount); future.completeExceptionally(new MetadataStoreException.BadVersionException("BadVersion")); @@ -1402,12 +1446,8 @@ public void splitAndRetryFailureTest() throws Exception { if (count.decrementAndGet() > 0) { return future; } - // Call the real method - reset(namespaceService); - doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) - .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); - return future; - }).when(namespaceService).updateNamespaceBundlesForPolicies(any(), any()); + return invocationOnMock.callRealMethod(); + }).when(namespaceService).updateNamespaceBundles(any(), any()); doReturn(namespaceService).when(pulsar1).getNamespaceService(); doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); @@ -1419,9 +1459,9 @@ public void splitAndRetryFailureTest() throws Exception { channel1.publishSplitEventAsync(split); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) @@ -1429,33 +1469,33 @@ public void splitAndRetryFailureTest() throws Exception { .untilAsserted(() -> { assertEquals(3, count.get()); }); - var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + ServiceUnitStateChannelImpl leader = + (ServiceUnitStateChannelImpl) (channel1.isChannelOwnerAsync().get() ? channel1 : channel2); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) .when(loadManager).selectAsync(any(), any(), any()); - ((ServiceUnitStateChannelImpl) leader) - .monitorOwnerships(List.of(brokerId1, brokerId2)); + leader.monitorOwnerships(List.of(brokerId1, brokerId2)); waitUntilState(leader, bundle3, Init); waitUntilState(channel1, bundle3, Init); waitUntilState(channel2, bundle3, Init); - - - validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 1, 0, 1, 0); - validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 1, 0, 1, 0); - validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); - validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); - waitUntilNewOwner(channel1, childBundle31, brokerId1); waitUntilNewOwner(channel1, childBundle32, brokerId1); waitUntilNewOwner(channel2, childBundle31, brokerId1); waitUntilNewOwner(channel2, childBundle32, brokerId1); + assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(childBundle31).get()); assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(childBundle32).get()); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(childBundle31).get()); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(childBundle32).get()); + validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 1, 0, 1, 0); + validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 1, 0, 1, 0); + validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); + validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); + + // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, @@ -1493,18 +1533,18 @@ public void splitAndRetryFailureTest() throws Exception { "stateTombstoneDelayTimeInMillis", 300 * 1000, true); } - @Test(priority = 17) + @Test(priority = 18) public void testOverrideInactiveBrokerStateData() throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { - var leaderChannel = channel1; - var followerChannel = channel2; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; + ServiceUnitStateChannelImpl followerChannel = (ServiceUnitStateChannelImpl) channel2; String leader = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader, leader2); if (leader.equals(brokerId2)) { - leaderChannel = channel2; - followerChannel = channel1; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; + followerChannel = (ServiceUnitStateChannelImpl) channel1; } String broker = brokerId1; @@ -1516,20 +1556,25 @@ public void testOverrideInactiveBrokerStateData() String freeBundle = "public/free/0xfffffff0_0xffffffff"; String deletedBundle = "public/deleted/0xfffffff0_0xffffffff"; String ownedBundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(releasingBundle, - new ServiceUnitStateData(Releasing, null, broker, 1)); - overrideTableViews(splittingBundle, - new ServiceUnitStateData(Splitting, null, broker, - Map.of(childBundle1Range, Optional.empty(), - childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle, - new ServiceUnitStateData(Assigning, broker, null, 1)); - overrideTableViews(freeBundle, - new ServiceUnitStateData(Free, null, broker, 1)); - overrideTableViews(deletedBundle, - new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle, - new ServiceUnitStateData(Owned, broker, null, 1)); + try { + disableChannels(); + overrideTableViews(releasingBundle, + new ServiceUnitStateData(Releasing, null, broker, 1)); + overrideTableViews(splittingBundle, + new ServiceUnitStateData(Splitting, null, broker, + Map.of(childBundle1Range, Optional.empty(), + childBundle2Range, Optional.empty()), 1)); + overrideTableViews(assigningBundle, + new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(freeBundle, + new ServiceUnitStateData(Free, null, broker, 1)); + overrideTableViews(deletedBundle, + new ServiceUnitStateData(Deleted, null, broker, 1)); + overrideTableViews(ownedBundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + } finally { + enableChannels(); + } // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) @@ -1558,7 +1603,7 @@ public void testOverrideInactiveBrokerStateData() } - @Test(priority = 18) + @Test(priority = 19) public void testOverrideOrphanStateData() throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { @@ -1586,28 +1631,33 @@ public void testOverrideOrphanStateData() String ownedBundle2 = "public/owned2SourceBundle/0xfffffff0_0xffffffff"; String ownedBundle3 = "public/owned3/0xfffffff0_0xffffffff"; String inactiveBroker = "broker-inactive-1"; - overrideTableViews(releasingBundle1, - new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); - overrideTableViews(releasingBundle2, - new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); - overrideTableViews(splittingBundle, - new ServiceUnitStateData(Splitting, null, broker, - Map.of(childBundle1Range, Optional.empty(), - childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle1, - new ServiceUnitStateData(Assigning, broker, null, 1)); - overrideTableViews(assigningBundle2, - new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); - overrideTableViews(freeBundle, - new ServiceUnitStateData(Free, null, broker, 1)); - overrideTableViews(deletedBundle, - new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle1, - new ServiceUnitStateData(Owned, broker, null, 1)); - overrideTableViews(ownedBundle2, - new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); - overrideTableViews(ownedBundle3, - new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + try { + disableChannels(); + overrideTableViews(releasingBundle1, + new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); + overrideTableViews(releasingBundle2, + new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); + overrideTableViews(splittingBundle, + new ServiceUnitStateData(Splitting, null, broker, + Map.of(childBundle1Range, Optional.empty(), + childBundle2Range, Optional.empty()), 1)); + overrideTableViews(assigningBundle1, + new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(assigningBundle2, + new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); + overrideTableViews(freeBundle, + new ServiceUnitStateData(Free, null, broker, 1)); + overrideTableViews(deletedBundle, + new ServiceUnitStateData(Deleted, null, broker, 1)); + overrideTableViews(ownedBundle1, + new ServiceUnitStateData(Owned, broker, null, 1)); + overrideTableViews(ownedBundle2, + new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); + overrideTableViews(ownedBundle3, + new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + } finally { + enableChannels(); + } // test stable metadata state @@ -1654,36 +1704,41 @@ public void testOverrideOrphanStateData() cleanTableViews(); } - @Test(priority = 19) + @Test(priority = 20) public void testActiveGetOwner() throws Exception { // case 1: the bundle owner is empty String broker = brokerId2; String bundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(bundle, null); - assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); - - // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner - overrideTableViews(bundle, - new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); - assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); - - - // case 3: the bundle ownership is transferring, and the dst broker is the channel owner - overrideTableViews(bundle, - new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); - assertTrue(!channel1.getOwnerAsync(bundle).isDone()); - - // case 4: the bundle ownership is found - overrideTableViews(bundle, - new ServiceUnitStateData(Owned, broker, null, 1)); - var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); - assertEquals(owner, broker); + try { + disableChannels(); + overrideTableViews(bundle, null); + assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); + + // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); + assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); + + + // case 3: the bundle ownership is transferring, and the dst broker is the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); + assertFalse(channel1.getOwnerAsync(bundle).isDone()); + + // case 4: the bundle ownership is found + overrideTableViews(bundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); + assertEquals(owner, broker); + } finally { + enableChannels(); + } // case 5: the owner lookup gets delayed var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", spyRegistry , true); + "brokerRegistry", spyRegistry, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); var delayedFuture = new CompletableFuture(); @@ -1692,7 +1747,7 @@ public void testActiveGetOwner() throws Exception { try { Thread.sleep(500); } catch (InterruptedException e) { - Thread.currentThread().interrupt();; + Thread.currentThread().interrupt(); } delayedFuture.complete(Optional.of(broker)); }); @@ -1716,12 +1771,12 @@ public void testActiveGetOwner() throws Exception { // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any(), any()); - var leaderChannel = channel1; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; String leader1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader1, leader2); if (leader1.equals(brokerId2)) { - leaderChannel = channel2; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; } leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", @@ -1739,8 +1794,13 @@ public void testActiveGetOwner() throws Exception { assertTrue(System.currentTimeMillis() - start < 20_000); // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel - overrideTableViews(bundle, - new ServiceUnitStateData(Owned, broker, null, 1)); + try { + disableChannels(); + overrideTableViews(bundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + } finally { + enableChannels(); + } doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); @@ -1758,12 +1818,12 @@ public void testActiveGetOwner() throws Exception { FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", registry , true); + "brokerRegistry", registry, true); cleanTableViews(); } - @Test(priority = 20) + @Test(priority = 21) public void testGetOwnershipEntrySetBeforeChannelStart() { var tmpChannel = new ServiceUnitStateChannelImpl(pulsar1); try { @@ -1775,6 +1835,33 @@ public void testGetOwnershipEntrySetBeforeChannelStart() { } } + @Test(priority = 22) + public void unloadTimeoutCheckTest() + throws Exception { + + String topic = "persistent://" + namespaceName + "/test-topic"; + NamespaceBundle bundleName = pulsar.getNamespaceService().getBundle(TopicName.get(topic)); + var releasing = new ServiceUnitStateData(Releasing, pulsar2.getBrokerId(), pulsar1.getBrokerId(), 1); + + try { + disableChannels(); + overrideTableView(channel1, bundleName.toString(), releasing); + var topicFuture = pulsar1.getBrokerService().getOrCreateTopic(topic); + topicFuture.get(1, TimeUnit.SECONDS); + } catch (Exception e) { + if (!(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException && e.getMessage() + .contains("Please redo the lookup"))) { + fail(); + } + } finally { + enableChannels(); + } + + pulsar1.getBrokerService() + .unloadServiceUnit(bundleName, true, true, 5, + TimeUnit.SECONDS).get(2, TimeUnit.SECONDS); + } + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { @@ -1846,10 +1933,21 @@ private static void waitUntilNewOwner(ServiceUnitStateChannel channel, String se }); } - private static void waitUntilState(ServiceUnitStateChannel channel, String key) + private static ServiceUnitStateTableView getTableView(ServiceUnitStateChannel channel) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) + return (ServiceUnitStateTableView) FieldUtils.readField(channel, "tableview", true); + } + + private static void setTableView(ServiceUnitStateChannel channel, + ServiceUnitStateTableView tableView) + throws IllegalAccessException { + FieldUtils.writeField(channel, "tableview", tableView, true); + } + + private static void waitUntilState(ServiceUnitStateChannel channel, String key) + throws IllegalAccessException { + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1865,8 +1963,7 @@ private static void waitUntilState(ServiceUnitStateChannel channel, String key) private static void waitUntilState(ServiceUnitStateChannel channel, String key, ServiceUnitState expected) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1879,8 +1976,7 @@ private static void waitUntilState(ServiceUnitStateChannel channel, String key, private void waitUntilStateWithMonitor(ServiceUnitStateChannel channel, String key, ServiceUnitState expected) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1893,28 +1989,50 @@ private void waitUntilStateWithMonitor(ServiceUnitStateChannel channel, String k }); } - private static void cleanTableView(ServiceUnitStateChannel channel, String serviceUnit) + private void cleanTableViews() throws IllegalAccessException { - var tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); - var cache = (ConcurrentMap) - FieldUtils.readField(tv, "data", true); - cache.remove(serviceUnit); + cleanTableView(channel1); + cleanTableView(channel2); } - private void cleanTableViews() - throws IllegalAccessException { - var tv1 = (TableViewImpl) - FieldUtils.readField(channel1, "tableview", true); - var cache1 = (ConcurrentMap) - FieldUtils.readField(tv1, "data", true); - cache1.clear(); - - var tv2 = (TableViewImpl) - FieldUtils.readField(channel2, "tableview", true); - var cache2 = (ConcurrentMap) - FieldUtils.readField(tv2, "data", true); - cache2.clear(); + private void cleanTableView(ServiceUnitStateChannel channel) throws IllegalAccessException { + var getOwnerRequests = (Map>) + FieldUtils.readField(channel, "getOwnerRequests", true); + getOwnerRequests.clear(); + var tv = getTableView(channel); + if (serviceUnitStateTableViewClassName.equals(ServiceUnitStateTableViewImpl.class.getCanonicalName())) { + var tableview = (TableView) + FieldUtils.readField(tv, "tableview", true); + var cache = (ConcurrentMap) + FieldUtils.readField(tableview, "data", true); + cache.clear(); + } else { + var tableview = (MetadataStoreTableView) + FieldUtils.readField(tv, "tableview", true); + var handlerCounters = + (Map) + FieldUtils.readDeclaredField(channel, "handlerCounters", true); + var initCounter = handlerCounters.get(Init).getTotal(); + var deletedCounter = new AtomicLong(initCounter.get()); + try { + var set = tableview.entrySet(); + for (var e : set) { + try { + tableview.delete(e.getKey()).join(); + deletedCounter.incrementAndGet(); + } catch (CompletionException ex) { + if (!(ex.getCause() instanceof MetadataStoreException.NotFoundException)) { + throw ex; + } + } + } + Awaitility.await().ignoreNoExceptions().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(initCounter.get(), deletedCounter.get()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } private void overrideTableViews(String serviceUnit, ServiceUnitStateData val) throws IllegalAccessException { @@ -1923,20 +2041,54 @@ private void overrideTableViews(String serviceUnit, ServiceUnitStateData val) th } @Test(enabled = false) - public static void overrideTableView(ServiceUnitStateChannel channel, String serviceUnit, ServiceUnitStateData val) - throws IllegalAccessException { - var tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + public static void overrideTableView(ServiceUnitStateChannel channel, + String serviceUnit, ServiceUnitStateData val) throws IllegalAccessException { var getOwnerRequests = (Map>) FieldUtils.readField(channel, "getOwnerRequests", true); - var cache = (ConcurrentMap) - FieldUtils.readField(tv, "data", true); - if(val == null){ - cache.remove(serviceUnit); - } else { - cache.put(serviceUnit, val); - } getOwnerRequests.clear(); + var tv = getTableView(channel); + + var handlerCounters = + (Map) + FieldUtils.readDeclaredField(channel, "handlerCounters", true); + + var cur = tv.get(serviceUnit); + if (cur != null) { + long intCountStart = handlerCounters.get(Init).getTotal().get(); + var deletedCount = new AtomicLong(0); + tv.delete(serviceUnit).join(); + deletedCount.incrementAndGet(); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals( + handlerCounters.get(Init).getTotal().get() + - intCountStart, deletedCount.get()); + assertNull(tv.get(serviceUnit)); + }); + } + + + + if (val != null) { + long stateCountStart = handlerCounters.get(state(val)).getTotal().get(); + var stateCount = new AtomicLong(0); + tv.put(serviceUnit, val).join(); + stateCount.incrementAndGet(); + + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals( + handlerCounters.get(state(val)).getTotal().get() + - stateCountStart, stateCount.get()); + assertEquals(val, tv.get(serviceUnit)); + }); + } + + } private static void cleanOpsCounters(ServiceUnitStateChannel channel) @@ -1945,7 +2097,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "handlerCounters", true); - for(var val : handlerCounters.values()){ + for (var val : handlerCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1954,7 +2106,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "eventCounters", true); - for(var val : eventCounters.values()){ + for (var val : eventCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1963,7 +2115,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "ownerLookUpCounters", true); - for(var val : ownerLookUpCounters.values()){ + for (var val : ownerLookUpCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1980,7 +2132,7 @@ private void cleanOwnershipMonitorCounters(ServiceUnitStateChannel channel) thro } private void cleanMetadataState(ServiceUnitStateChannel channel) throws IllegalAccessException { - channel.handleMetadataSessionEvent(SessionReestablished); + ((ServiceUnitStateChannelImpl) channel).handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(channel, "lastMetadataSessionEventTimestamp", 0L, true); } @@ -2058,7 +2210,7 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, long free, long deleted, long init - ) + ) throws IllegalAccessException { var ownerLookUpCounters = (Map) @@ -2126,4 +2278,14 @@ ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) return channel; } + + private void disableChannels() { + ((ServiceUnitStateChannelImpl) channel1).disable(); + ((ServiceUnitStateChannelImpl) channel2).disable(); + } + + private void enableChannels() { + ((ServiceUnitStateChannelImpl) channel1).enable(); + ((ServiceUnitStateChannelImpl) channel2).enable(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java similarity index 88% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java index 049da191a80ab..d336e8918ec5e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java @@ -25,13 +25,15 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import org.testng.annotations.Test; @Test(groups = "broker") -public class ServiceUnitStateCompactionStrategyTest { - ServiceUnitStateCompactionStrategy strategy = new ServiceUnitStateCompactionStrategy(); +public class ServiceUnitStateDataConflictResolverTest { + ServiceUnitStateDataConflictResolver strategy = new ServiceUnitStateDataConflictResolver(); String dst = "dst"; String src = "src"; @@ -91,6 +93,32 @@ public void testVersionId(){ } + @Test + public void testStoreType(){ + ServiceUnitStateDataConflictResolver strategy = new ServiceUnitStateDataConflictResolver(); + strategy.setStorageType(SystemTopic); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 1))); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 2))); + assertFalse(strategy.shouldKeepLeft( + new ServiceUnitStateData(Owned, dst, 1), + new ServiceUnitStateData(Owned, dst, 3))); + + strategy.setStorageType(MetadataStore); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 1))); + assertTrue(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 2))); + assertTrue(strategy.shouldKeepLeft( + new ServiceUnitStateData(Owned, dst, 1), + new ServiceUnitStateData(Owned, dst, 3))); + } + @Test public void testForce(){ assertFalse(strategy.shouldKeepLeft( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java index 620266aee46a1..0a5f012ad40a7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java @@ -25,6 +25,8 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import org.testng.annotations.Test; @@ -54,63 +56,123 @@ public void testActive() { } @Test - public void testTransitions() { - - assertFalse(ServiceUnitState.isValidTransition(Init, Init)); - assertTrue(ServiceUnitState.isValidTransition(Init, Free)); - assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Init, Assigning)); - assertTrue(ServiceUnitState.isValidTransition(Init, Releasing)); - assertTrue(ServiceUnitState.isValidTransition(Init, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Init, Deleted)); - - assertTrue(ServiceUnitState.isValidTransition(Free, Init)); - assertFalse(ServiceUnitState.isValidTransition(Free, Free)); - assertFalse(ServiceUnitState.isValidTransition(Free, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Free, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Free, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Free, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Assigning, Init)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Free)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning)); - assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Owned, Init)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Free)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Releasing, Init)); - assertTrue(ServiceUnitState.isValidTransition(Releasing, Free)); - assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Splitting, Init)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted)); - - assertTrue(ServiceUnitState.isValidTransition(Deleted, Init)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Free)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted)); + public void testTransitionsOverSystemTopic() { + + assertFalse(ServiceUnitState.isValidTransition(Init, Init, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Free, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigning, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Releasing, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Splitting, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Deleted, SystemTopic)); + + assertTrue(ServiceUnitState.isValidTransition(Free, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Assigning, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Owned, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Releasing, Init, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Free, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Splitting, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted, SystemTopic)); + + assertTrue(ServiceUnitState.isValidTransition(Deleted, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted, SystemTopic)); + } + + @Test + public void testTransitionsOverMetadataStore() { + + assertFalse(ServiceUnitState.isValidTransition(Init, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Free, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Assigning, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Owned, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Releasing, Init, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Free, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Splitting, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted, MetadataStore)); + + assertTrue(ServiceUnitState.isValidTransition(Deleted, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted, MetadataStore)); } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java index d25cba2bd1bdd..3267e67ad2c3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java @@ -18,8 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.store; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertThrows; import static org.testng.AssertJUnit.assertTrue; import com.google.common.collect.Sets; @@ -33,6 +37,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -75,8 +80,6 @@ public void testPushGetAndRemove() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, MyClass.class); - loadDataStore.startProducer(); - loadDataStore.startTableView(); MyClass myClass1 = new MyClass("1", 1); loadDataStore.pushAsync("key1", myClass1).get(); @@ -109,8 +112,6 @@ public void testForEach() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, Integer.class); - loadDataStore.startProducer(); - loadDataStore.startTableView(); Map map = new HashMap<>(); for (int i = 0; i < 10; i++) { @@ -134,9 +135,6 @@ public void testTableViewRestart() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, Integer.class); - loadDataStore.startProducer(); - - loadDataStore.startTableView(); loadDataStore.pushAsync("1", 1).get(); Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.size(), 1)); assertEquals(loadDataStore.get("1").get(), 1); @@ -150,6 +148,31 @@ public void testTableViewRestart() throws Exception { Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 3)); } + @Test + public void testProducerRestart() throws Exception { + String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); + var loadDataStore = + (TableViewLoadDataStoreImpl) spy(LoadDataStoreFactory.create(pulsar, topic, Integer.class)); + + // happy case + loadDataStore.pushAsync("1", 1).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.size(), 1)); + assertEquals(loadDataStore.get("1").get(), 1); + verify(loadDataStore, times(1)).startProducer(); + + // loadDataStore will restart producer if null. + FieldUtils.writeField(loadDataStore, "producer", null, true); + loadDataStore.pushAsync("1", 2).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 2)); + verify(loadDataStore, times(2)).startProducer(); + + // loadDataStore will restart producer if too slow. + FieldUtils.writeField(loadDataStore, "producerLastPublishTimestamp", 0 , true); + loadDataStore.pushAsync("1", 3).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 3)); + verify(loadDataStore, times(3)).startProducer(); + } + @Test public void testProducerStop() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); @@ -165,4 +188,26 @@ public void testProducerStop() throws Exception { loadDataStore.removeAsync("2").get(); } + @Test + public void testShutdown() throws Exception { + String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); + LoadDataStore loadDataStore = + LoadDataStoreFactory.create(pulsar, topic, Integer.class); + loadDataStore.start(); + loadDataStore.shutdown(); + + Assert.assertTrue(loadDataStore.pushAsync("2", 2).isCompletedExceptionally()); + Assert.assertTrue(loadDataStore.removeAsync("2").isCompletedExceptionally()); + assertThrows(IllegalStateException.class, () -> loadDataStore.get("2")); + assertThrows(IllegalStateException.class, loadDataStore::size); + assertThrows(IllegalStateException.class, loadDataStore::entrySet); + assertThrows(IllegalStateException.class, () -> loadDataStore.forEach((k, v) -> {})); + assertThrows(IllegalStateException.class, loadDataStore::init); + assertThrows(IllegalStateException.class, loadDataStore::start); + assertThrows(IllegalStateException.class, loadDataStore::startProducer); + assertThrows(IllegalStateException.class, loadDataStore::startTableView); + assertThrows(IllegalStateException.class, loadDataStore::close); + assertThrows(IllegalStateException.class, loadDataStore::closeTableView); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index ea5365bcf4b2c..a4460187d2377 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -32,7 +33,6 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; @@ -547,7 +547,7 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() tenantInfo.setAllowedClusters(Set.of(configClusterName)); admin.tenants().createTenant("pulsar", tenantInfo); admin.namespaces().createNamespace(namespaceName); - admin.topics().createNonPartitionedTopic(ServiceUnitStateChannelImpl.TOPIC); + admin.topics().createNonPartitionedTopic(TOPIC); admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC); admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); @@ -560,7 +560,7 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() // The created persistent topic correctly can be found by // pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - Producer producer = pulsarClient.newProducer().topic(ServiceUnitStateChannelImpl.TOPIC).create(); + Producer producer = pulsarClient.newProducer().topic(TOPIC).create(); // The created non-persistent topics cannot be found, as we did topics.clear() try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 2f27d5917f025..5398b5aa57b8b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_LOG; import static org.mockito.ArgumentMatchers.anyString; @@ -82,7 +83,6 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -1759,7 +1759,7 @@ public void testMetricsNonPersistentTopicLoadFails() throws Exception { public void testIsSystemTopicAllowAutoTopicCreationAsync() throws Exception { BrokerService brokerService = pulsar.getBrokerService(); assertFalse(brokerService.isAllowAutoTopicCreationAsync( - ServiceUnitStateChannelImpl.TOPIC).get()); + TOPIC).get()); assertTrue(brokerService.isAllowAutoTopicCreationAsync( "persistent://pulsar/system/my-system-topic").get()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java index 9140216810826..a834fa1fde1e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java @@ -24,9 +24,10 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isValidTransition; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MSG_COMPRESSION_TYPE; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.MSG_COMPRESSION_TYPE; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -61,7 +62,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateDataConflictResolver; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -92,7 +93,7 @@ public class ServiceUnitStateCompactionTest extends MockedPulsarServiceBaseTest private ScheduledExecutorService compactionScheduler; private BookKeeper bk; private Schema schema; - private ServiceUnitStateCompactionStrategy strategy; + private ServiceUnitStateDataConflictResolver strategy; private ServiceUnitState testState = Init; @@ -118,7 +119,7 @@ private ServiceUnitStateData testValue(String broker) { private ServiceUnitState nextValidState(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> isValidTransition(from, to)) + .filter(to -> isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; @@ -127,7 +128,7 @@ private ServiceUnitState nextValidState(ServiceUnitState from) { private ServiceUnitState nextValidStateNonSplit(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) .filter(to -> to != Init && to != Splitting && to != Deleted - && isValidTransition(from, to)) + && isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; @@ -135,7 +136,7 @@ && isValidTransition(from, to)) private ServiceUnitState nextInvalidState(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> !isValidTransition(from, to)) + .filter(to -> !isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); if (candidates.size() == 0) { return Init; @@ -157,7 +158,7 @@ public void setup() throws Exception { new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); schema = Schema.JSON(ServiceUnitStateData.class); - strategy = new ServiceUnitStateCompactionStrategy(); + strategy = new ServiceUnitStateDataConflictResolver(); strategy.checkBrokers(false); testState = Init; @@ -329,10 +330,10 @@ public void testCompactionWithTableview() throws Exception { .topic("persistent://my-property/use/my-ns/my-topic1") .loadConf(Map.of( "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); - ((ServiceUnitStateCompactionStrategy) + ((ServiceUnitStateDataConflictResolver) FieldUtils.readDeclaredField(tv, "compactionStrategy", true)) .checkBrokers(false); TestData testData = generateTestData(); @@ -364,7 +365,7 @@ public void testCompactionWithTableview() throws Exception { .topic(topic) .loadConf(Map.of( "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); for(var etr : tableview.entrySet()){ @@ -531,7 +532,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .subscriptionName("fastTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); var defaultConf = getDefaultConf(); @@ -544,7 +545,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .subscriptionName("slowTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); var semaphore = new Semaphore(0); @@ -616,7 +617,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .topic(topic) .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) @@ -651,7 +652,7 @@ public void testSlowReceiveTableviewAfterCompaction() throws Exception { .subscriptionName("slowTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); // Configure retention to ensue data is retained for reader diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java index d1ff46cbc02d5..8f67e412267a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.compaction; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MSG_COMPRESSION_TYPE; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.MSG_COMPRESSION_TYPE; import static org.testng.Assert.assertEquals; import java.util.ArrayList; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java index 55b159071fda4..2bc042aebb308 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.metadata.api; +import java.util.Optional; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import lombok.Builder; import lombok.Getter; import lombok.ToString; @@ -29,7 +31,7 @@ @Builder @Getter @ToString -public class MetadataCacheConfig { +public class MetadataCacheConfig { private static final long DEFAULT_CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5); /** @@ -47,4 +49,12 @@ public class MetadataCacheConfig { */ @Builder.Default private final long expireAfterWriteMillis = 2 * DEFAULT_CACHE_REFRESH_TIME_MILLIS; + + /** + * Specifies cache reload consumer behavior when the cache is refreshed automatically at refreshAfterWriteMillis + * frequency. + */ + @Builder.Default + private final BiConsumer>> asyncReloadConsumer = null; + } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java new file mode 100644 index 0000000000000..64de22890a0f1 --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java @@ -0,0 +1,87 @@ +/* + * 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.metadata.api; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** + * Defines metadata store tableview. + * MetadataStoreTableView initially fills existing items to its local tableview and eventually + * synchronize remote updates to its local tableview from the remote metadata store. + * This abstraction can help replicate metadata in memory from metadata store. + */ +public interface MetadataStoreTableView { + + class ConflictException extends RuntimeException { + public ConflictException(String msg) { + super(msg); + } + } + + /** + * Starts the tableview by filling existing items to its local tableview from the remote metadata store. + */ + void start() throws MetadataStoreException; + + /** + * Gets one item from the local tableview. + *

+ * If the key is not found, return null. + * + * @param key the key to check + * @return value if exists. Otherwise, null. + */ + T get(String key); + + /** + * Tries to put the item in the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this put value. + *

+ * This operation can fail if the input value conflicts with the existing one. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreTableView.ConflictException + * if the input value conflicts with the existing one. + */ + CompletableFuture put(String key, T value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + *

+ * This can fail if the item is not present in the metadata store. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreException.NotFoundException + * if the key is not present in the metadata store. + */ + CompletableFuture delete(String key); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); +} + diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index ee394b0267c88..4c7f34aa5c16e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -61,15 +61,15 @@ public class MetadataCacheImpl implements MetadataCache, Consumer>> objCache; - public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig) { this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig); } - public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig) { this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig); } - public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { this.store = store; if (store instanceof MetadataStoreExtended) { this.storeExtended = (MetadataStoreExtended) store; @@ -98,7 +98,12 @@ public CompletableFuture>> asyncReload( Optional> oldValue, Executor executor) { if (store instanceof AbstractMetadataStore && ((AbstractMetadataStore) store).isConnected()) { - return readValueFromStore(key); + return readValueFromStore(key).thenApply(val -> { + if (cacheConfig.getAsyncReloadConsumer() != null) { + cacheConfig.getAsyncReloadConsumer().accept(key, val); + } + return val; + }); } else { // Do not try to refresh the cache item if we know that we're not connected to the // metadata store diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java new file mode 100644 index 0000000000000..4f9aad0ba658b --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java @@ -0,0 +1,342 @@ +/* + * 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.metadata.tableview.impl; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.BiConsumer; +import java.util.function.BiPredicate; +import java.util.function.Predicate; +import lombok.Builder; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.CacheGetResult; +import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; +import org.apache.pulsar.metadata.api.NotificationType; + +@Slf4j +public class MetadataStoreTableViewImpl implements MetadataStoreTableView { + + private static final int FILL_TIMEOUT_IN_MILLIS = 300_000; + private static final int MAX_CONCURRENT_METADATA_OPS_DURING_FILL = 50; + private static final long CACHE_REFRESH_FREQUENCY_IN_MILLIS = 600_000; + private final ConcurrentMap data; + private final Map immutableData; + private final String name; + private final MetadataStore store; + private final MetadataCache cache; + private final Predicate listenPathValidator; + private final BiPredicate conflictResolver; + private final List> tailItemListeners; + private final List> existingItemListeners; + private final long timeoutInMillis; + private final String pathPrefix; + + /** + * Construct MetadataStoreTableViewImpl. + * + * @param clazz clazz of the value type + * @param name metadata store tableview name + * @param store metadata store + * @param pathPrefix metadata store path prefix + * @param listenPathValidator path validator to listen + * @param conflictResolver resolve conflicts for concurrent puts + * @param tailItemListeners listener for tail item(recently updated) notifications + * @param existingItemListeners listener for existing items in metadata store + * @param timeoutInMillis timeout duration for each sync operation. + * @throws MetadataStoreException if init fails. + */ + @Builder + public MetadataStoreTableViewImpl(@NonNull Class clazz, + @NonNull String name, + @NonNull MetadataStore store, + @NonNull String pathPrefix, + @NonNull BiPredicate conflictResolver, + Predicate listenPathValidator, + List> tailItemListeners, + List> existingItemListeners, + long timeoutInMillis) { + this.name = name; + this.data = new ConcurrentHashMap<>(); + this.immutableData = Collections.unmodifiableMap(data); + this.pathPrefix = pathPrefix; + this.conflictResolver = conflictResolver; + this.listenPathValidator = listenPathValidator; + this.tailItemListeners = new ArrayList<>(); + if (tailItemListeners != null) { + this.tailItemListeners.addAll(tailItemListeners); + } + this.existingItemListeners = new ArrayList<>(); + if (existingItemListeners != null) { + this.existingItemListeners.addAll(existingItemListeners); + } + this.timeoutInMillis = timeoutInMillis; + this.store = store; + this.cache = store.getMetadataCache(clazz, + MetadataCacheConfig.builder() + .expireAfterWriteMillis(-1) + .refreshAfterWriteMillis(CACHE_REFRESH_FREQUENCY_IN_MILLIS) + .asyncReloadConsumer(this::consumeAsyncReload) + .build()); + store.registerListener(this::handleNotification); + } + + public void start() throws MetadataStoreException { + fill(); + } + + private void consumeAsyncReload(String path, Optional> cached) { + if (!isValidPath(path)) { + return; + } + String key = getKey(path); + var val = getValue(cached); + handleTailItem(key, val); + } + + private boolean isValidPath(String path) { + if (listenPathValidator != null && !listenPathValidator.test(path)) { + return false; + } + return true; + } + + private T getValue(Optional> cached) { + return cached.map(CacheGetResult::getValue).orElse(null); + } + + boolean updateData(String key, T cur) { + MutableBoolean updated = new MutableBoolean(); + data.compute(key, (k, prev) -> { + if (Objects.equals(prev, cur)) { + if (log.isDebugEnabled()) { + log.debug("{} skipped item key={} value={} prev={}", + name, key, cur, prev); + } + updated.setValue(false); + return prev; + } else { + updated.setValue(true); + return cur; + } + }); + return updated.booleanValue(); + } + + private void handleTailItem(String key, T val) { + if (updateData(key, val)) { + if (log.isDebugEnabled()) { + log.debug("{} applying item key={} value={}", + name, + key, + val); + } + for (var listener : tailItemListeners) { + try { + listener.accept(key, val); + } catch (Throwable e) { + log.error("{} failed to listen tail item key:{}, val:{}", + name, + key, val, e); + } + } + } + + } + + private CompletableFuture doHandleNotification(String path) { + if (!isValidPath(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path).thenAccept(valOpt -> { + String key = getKey(path); + var val = valOpt.orElse(null); + handleTailItem(key, val); + }).exceptionally(e -> { + log.error("{} failed to handle notification for path:{}", name, path, e); + return null; + }); + } + + private void handleNotification(org.apache.pulsar.metadata.api.Notification notification) { + + if (notification.getType() == NotificationType.ChildrenChanged) { + return; + } + + String path = notification.getPath(); + + doHandleNotification(path); + } + + + private CompletableFuture handleExisting(String path) { + if (!isValidPath(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path) + .thenAccept(valOpt -> { + valOpt.ifPresent(val -> { + String key = getKey(path); + updateData(key, val); + if (log.isDebugEnabled()) { + log.debug("{} applying existing item key={} value={}", + name, + key, + val); + } + for (var listener : existingItemListeners) { + try { + listener.accept(key, val); + } catch (Throwable e) { + log.error("{} failed to listen existing item key:{}, val:{}", name, key, val, + e); + throw e; + } + } + }); + }); + } + + private void fill() throws MetadataStoreException { + final var deadline = System.currentTimeMillis() + FILL_TIMEOUT_IN_MILLIS; + log.info("{} start filling existing items under the pathPrefix:{}", name, pathPrefix); + ConcurrentLinkedDeque q = new ConcurrentLinkedDeque<>(); + List> futures = new ArrayList<>(); + q.add(pathPrefix); + LongAdder count = new LongAdder(); + while (!q.isEmpty()) { + var now = System.currentTimeMillis(); + if (now >= deadline) { + String err = name + " failed to fill existing items in " + + TimeUnit.MILLISECONDS.toSeconds(FILL_TIMEOUT_IN_MILLIS) + " secs. Filled count:" + + count.sum(); + log.error(err); + throw new MetadataStoreException(err); + } + int size = Math.min(MAX_CONCURRENT_METADATA_OPS_DURING_FILL, q.size()); + for (int i = 0; i < size; i++) { + String path = q.poll(); + futures.add(store.getChildren(path) + .thenCompose(children -> { + // The path is leaf + if (children.isEmpty()) { + count.increment(); + return handleExisting(path); + } else { + for (var child : children) { + q.add(path + "/" + child); + } + return CompletableFuture.completedFuture(null); + } + })); + } + try { + FutureUtil.waitForAll(futures).get( + Math.min(timeoutInMillis, deadline - now), + TimeUnit.MILLISECONDS); + } catch (Throwable e) { + Throwable c = FutureUtil.unwrapCompletionException(e); + log.error("{} failed to fill existing items", name, c); + throw new MetadataStoreException(c); + } + futures.clear(); + } + log.info("{} completed filling existing items with size:{}", name, count.sum()); + } + + + private String getPath(String key) { + return pathPrefix + "/" + key; + } + + private String getKey(String path) { + return path.replaceFirst(pathPrefix + "/", ""); + } + + public boolean exists(String key) { + return immutableData.containsKey(key); + } + + public T get(String key) { + return data.get(key); + } + + public CompletableFuture put(String key, T value) { + String path = getPath(key); + return cache.readModifyUpdateOrCreate(path, (old) -> { + if (conflictResolver.test(old.orElse(null), value)) { + return value; + } else { + throw new ConflictException( + String.format("Failed to update from old:%s to value:%s", old, value)); + } + }).thenCompose(__ -> doHandleNotification(path)); // immediately notify local tableview + } + + public CompletableFuture delete(String key) { + String path = getPath(key); + return cache.delete(path) + .thenCompose(__ -> doHandleNotification(path)); // immediately notify local tableview + } + + public int size() { + return immutableData.size(); + } + + public boolean isEmpty() { + return immutableData.isEmpty(); + } + + public Set> entrySet() { + return immutableData.entrySet(); + } + + public Set keySet() { + return immutableData.keySet(); + } + + public Collection values() { + return immutableData.values(); + } + + public void forEach(BiConsumer action) { + immutableData.forEach(action); + } + +} diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java new file mode 100644 index 0000000000000..2c47770610b05 --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.metadata.tableview.impl; diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index bac5807360453..6992c69b7252e 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.api.CacheGetResult; import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataSerde; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -622,4 +623,28 @@ public void testPut(String provider, Supplier urlSupplier) throws Except assertEquals(cache2.get(key).get().orElse(-1), 2); }); } + + @Test(dataProvider = "impl") + public void testAsyncReloadConsumer(String provider, Supplier urlSupplier) throws Exception { + @Cleanup + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + + List refreshed = new ArrayList<>(); + MetadataCache objCache = store.getMetadataCache(MyClass.class, + MetadataCacheConfig.builder().refreshAfterWriteMillis(100) + .asyncReloadConsumer((k, v) -> v.map(vv -> refreshed.add(vv.getValue()))).build()); + + String key1 = newKey(); + + MyClass value1 = new MyClass("a", 1); + objCache.create(key1, value1); + + MyClass value2 = new MyClass("a", 2); + store.put(key1, ObjectMapperFactory.getMapper().writer().writeValueAsBytes(value2), Optional.empty()) + .join(); + + Awaitility.await().untilAsserted(() -> { + refreshed.contains(value2); + }); + } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java new file mode 100644 index 0000000000000..5a2ea32890dbd --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java @@ -0,0 +1,499 @@ +/* + * 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.metadata; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import com.fasterxml.jackson.databind.type.TypeFactory; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import lombok.Cleanup; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; +import org.apache.pulsar.metadata.cache.impl.JSONMetadataSerdeSimpleType; +import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; +import org.apache.pulsar.metadata.tableview.impl.MetadataStoreTableViewImpl; +import org.awaitility.Awaitility; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Slf4j +public class MetadataStoreTableViewTest extends BaseMetadataStoreTest { + + LinkedBlockingDeque> tails; + LinkedBlockingDeque> existings; + + @BeforeMethod + void init(){ + tails = new LinkedBlockingDeque<>(); + existings = new LinkedBlockingDeque<>(); + } + + private void tailListener(String k, Integer v){ + tails.add(Pair.of(k, v)); + } + + private void existingListener(String k, Integer v){ + existings.add(Pair.of(k, v)); + } + + MetadataStoreTableViewImpl createTestTableView(MetadataStore store, String prefix, + Supplier urlSupplier) + throws Exception { + var tv = MetadataStoreTableViewImpl.builder() + .name("test") + .clazz(Integer.class) + .store(store) + .pathPrefix(prefix) + .conflictResolver((old, cur) -> { + if (old == null || cur == null) { + return true; + } + return old < cur; + }) + .listenPathValidator((path) -> path.startsWith(prefix) && path.contains("my")) + .tailItemListeners(List.of(this::tailListener)) + .existingItemListeners(List.of(this::existingListener)) + .timeoutInMillis(5_000) + .build(); + tv.start(); + return tv; + } + + private void assertGet(MetadataStoreTableViewImpl tv, String path, Integer expected) { + assertEquals(tv.get(path), expected); + //Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> assertEquals(tv.get(path), expected)); + } + + + @Test(dataProvider = "impl") + public void emptyTableViewTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + assertFalse(tv.exists("non-existing-key")); + assertFalse(tv.exists("non-existing-key/child")); + assertNull(tv.get("non-existing-key")); + assertNull(tv.get("non-existing-key/child")); + + try { + tv.delete("non-existing-key").join(); + fail("Should have failed"); + } catch (CompletionException e) { + assertException(e, NotFoundException.class); + } + + } + + @Test(dataProvider = "impl") + public void concurrentPutTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + int data = 1; + String path = "my"; + int concurrent = 50; + List> futureList = new ArrayList<>(); + for (int i = 0; i < concurrent; i++) { + futureList.add(tv.put(path, data).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + })); + } + FutureUtil.waitForAll(futureList).join(); + + assertGet(tv, path, data); + } + + @Test(dataProvider = "impl") + public void conflictResolverTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key1 = "my"; + + tv.put(key1, 0).join(); + tv.put(key1, 0).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + }).join(); + assertGet(tv, key1, 0); + tv.put(key1, 1).join(); + assertGet(tv, key1, 1); + tv.put(key1, 0).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + }).join(); + assertGet(tv, key1, 1); + } + + @Test(dataProvider = "impl") + public void deleteTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key1 = "key"; + tv.put(key1, 0).join(); + tv.delete(key1).join(); + assertNull(tv.get(key1)); + } + + @Test(dataProvider = "impl") + public void mapApiTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + assertTrue(tv.isEmpty()); + assertEquals(tv.size(), 0); + + String key1 = "my1"; + String key2 = "my2"; + + int val1 = 1; + int val2 = 2; + + tv.put(key1, val1).join(); + tv.put(key2, val2).join(); + assertGet(tv, key1, 1); + assertGet(tv, key2, 2); + + assertFalse(tv.isEmpty()); + assertEquals(tv.size(), 2); + + List actual = new ArrayList<>(); + tv.forEach((k, v) -> { + actual.add(k + "," + v); + }); + assertEquals(actual, List.of(key1 + "," + val1, key2 + "," + val2)); + + var values = tv.values(); + assertEquals(values.size(), 2); + assertTrue(values.containsAll(List.of(val1, val2))); + + var keys = tv.keySet(); + assertEquals(keys.size(), 2); + assertTrue(keys.containsAll(List.of(key1, key2))); + + var entries = tv.entrySet(); + assertEquals(entries.size(), 2); + assertTrue(entries.containsAll(Map.of(key1, val1, key2, val2).entrySet())); + } + + @Test(dataProvider = "impl") + public void notificationListeners(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String keyPrefix = "tenant/ns"; + String key1 = keyPrefix + "/my-1"; + int val1 = 1; + + assertGet(tv, key1, null); + + // listen on put + tv.put(key1, val1).join(); + var kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, val1)); + assertEquals(tv.get(key1), val1); + + // listen on modified + int val2 = 2; + tv.put(key1, val2).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, val2)); + assertEquals(tv.get(key1), val2); + + // no listen on the parent + int val0 = 0; + String childKey = key1 + "/my-child-1"; + tv.put(childKey, val0).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, val0)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertEquals(tv.get(childKey), val0); + + tv.put(childKey, val1).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, val1)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertEquals(tv.get(childKey), val1); + + tv.delete(childKey).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, null)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertNull(tv.get(childKey)); + + // No listen on the filtered key + String noListenKey = keyPrefix + "/to-be-filtered"; + tv.put(noListenKey, val0).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertNull(tv.get(noListenKey)); + + // Trigger deleted notification + tv.delete(key1).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, null)); + assertNull(tv.get(key1)); + } + + @Test(dataProvider = "impl") + public void testConcurrentPutGetOneKey(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key = "my"; + int val = 0; + int maxValue = 50; + tv.put(key, val).join(); + + AtomicInteger successWrites = new AtomicInteger(0); + Runnable task = new Runnable() { + @SneakyThrows + @Override + public void run() { + for (int k = 0; k < 1000; k++) { + var kv = tails.poll(3, TimeUnit.SECONDS); + if (kv == null) { + break; + } + Integer val = kv.getRight() + 1; + if (val <= maxValue) { + CompletableFuture putResult = + tv.put(key, val).thenRun(successWrites::incrementAndGet); + try { + putResult.get(); + } catch (Exception ignore) { + } + log.info("Put value {} success:{}. ", val, !putResult.isCompletedExceptionally()); + } else { + break; + } + } + } + }; + CompletableFuture t1 = CompletableFuture.completedFuture(null).thenRunAsync(task); + CompletableFuture t2 = CompletableFuture.completedFuture(null).thenRunAsync(task); + task.run(); + t1.join(); + t2.join(); + assertFalse(t1.isCompletedExceptionally()); + assertFalse(t2.isCompletedExceptionally()); + + assertEquals(successWrites.get(), maxValue); + assertEquals(tv.get(key), maxValue); + } + + @Test(dataProvider = "impl") + public void testConcurrentPut(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String k = "my"; + int v = 0; + CompletableFuture f1 = + CompletableFuture.runAsync(() -> tv.put(k, v).join()); + CompletableFuture f2 = + CompletableFuture.runAsync(() -> tv.put(k, v).join()); + Awaitility.await().until(() -> f1.isDone() && f2.isDone()); + assertTrue(f1.isCompletedExceptionally() && !f2.isCompletedExceptionally() || + ! f1.isCompletedExceptionally() && f2.isCompletedExceptionally()); + } + + @Test(dataProvider = "impl") + public void testConcurrentDelete(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String k = "my"; + tv.put(k, 0).join(); + CompletableFuture f1 = + CompletableFuture.runAsync(() -> tv.delete(k).join()); + CompletableFuture f2 = + CompletableFuture.runAsync(() -> tv.delete(k).join()); + Awaitility.await().until(() -> f1.isDone() && f2.isDone()); + assertTrue(f1.isCompletedExceptionally() && !f2.isCompletedExceptionally() || + ! f1.isCompletedExceptionally() && f2.isCompletedExceptionally()); + } + + @Test(dataProvider = "impl") + public void testClosedMetadataStore(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + String k = "my"; + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + store.close(); + try { + tv.put(k, 0).get(); + fail(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MetadataStoreException.AlreadyClosedException); + } + try { + tv.delete(k).get(); + fail(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MetadataStoreException.AlreadyClosedException); + } + + } + + + @Test(dataProvider = "distributedImpl") + public void testGetIfCachedDistributed(String provider, Supplier urlSupplier) throws Exception { + + String prefix = newKey(); + String k = "my"; + @Cleanup + MetadataStore store1 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv1 = createTestTableView(store1, prefix, urlSupplier); + @Cleanup + MetadataStore store2 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv2 = createTestTableView(store2, prefix, urlSupplier); + + + assertNull(tv1.get(k)); + assertNull(tv2.get(k)); + + tv1.put(k, 0).join(); + assertGet(tv1, k, 0); + Awaitility.await() + .untilAsserted(() -> assertEquals(tv2.get(k), 0)); + + tv2.put(k, 1).join(); + assertGet(tv2, k, 1); + Awaitility.await() + .untilAsserted(() -> assertEquals(tv1.get(k), 1)); + + tv1.delete(k).join(); + assertGet(tv1, k, null); + Awaitility.await() + .untilAsserted(() -> assertNull(tv2.get(k))); + } + + @Test(dataProvider = "distributedImpl") + public void testInitialFill(String provider, Supplier urlSupplier) throws Exception { + + String prefix = newKey(); + String k1 = "tenant-1/ns-1/my-1"; + String k2 = "tenant-1/ns-1/my-2"; + String k3 = "tenant-1/ns-2/my-3"; + String k4 = "tenant-2/ns-3/my-4"; + String k5 = "tenant-2/ns-3/your-1"; + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl btv = createTestTableView(store, prefix, urlSupplier); + + assertFalse(btv.exists(k1)); + + var serde = new JSONMetadataSerdeSimpleType<>( + TypeFactory.defaultInstance().constructSimpleType(Integer.class, null)); + store.put(prefix + "/" + k1, serde.serialize(prefix + "/" + k1, 0), Optional.empty()).join(); + store.put(prefix + "/" + k2, serde.serialize(prefix + "/" + k2, 1), Optional.empty()).join(); + store.put(prefix + "/" + k3, serde.serialize(prefix + "/" + k3, 2), Optional.empty()).join(); + store.put(prefix + "/" + k4, serde.serialize(prefix + "/" + k4, 3), Optional.empty()).join(); + store.put(prefix + "/" + k5, serde.serialize(prefix + "/" + k5, 4), Optional.empty()).join(); + + var expected = new HashSet<>(Set.of(Pair.of(k1, 0), Pair.of(k2, 1), Pair.of(k3, 2), Pair.of(k4, 3))); + var tailExpected = new HashSet<>(expected); + + for (int i = 0; i < 4; i++) { + var kv = tails.poll(3, TimeUnit.SECONDS); + assertTrue(tailExpected.remove(kv)); + } + assertNull(tails.poll(3, TimeUnit.SECONDS)); + assertTrue(tailExpected.isEmpty()); + + @Cleanup + MetadataStore store2 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store2, prefix, urlSupplier); + + var existingExpected = new HashSet<>(Set.of(Pair.of(k1, 0), Pair.of(k2, 1), Pair.of(k3, 2), Pair.of(k4, 3))); + var entrySetExpected = expected.stream().collect(Collectors.toMap(Pair::getLeft, Pair::getRight)).entrySet(); + + + for (int i = 0; i < 4; i++) { + var kv = existings.poll(3, TimeUnit.SECONDS); + assertTrue(existingExpected.remove(kv)); + } + assertNull(existings.poll(3, TimeUnit.SECONDS)); + assertTrue(existingExpected.isEmpty()); + + assertEquals(tv.get(k1), 0); + assertEquals(tv.get(k2), 1); + assertEquals(tv.get(k3), 2); + assertEquals(tv.get(k4), 3); + assertNull(tv.get(k5)); + + assertEquals(tv.entrySet(), entrySetExpected); + } +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index ee7497010adfc..759e689b24d0f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -57,6 +57,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @@ -78,6 +80,20 @@ public class ExtensibleLoadManagerTest extends TestRetrySupport { private PulsarCluster pulsarCluster = null; private String hosts; private PulsarAdmin admin; + protected String serviceUnitStateTableViewClassName; + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerTest(String serviceUnitStateTableViewClassName) { + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; + } + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {"org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl"}, + {"org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl"} + }; + } @BeforeClass(alwaysRun = true) public void setup() throws Exception { @@ -87,6 +103,8 @@ public void setup() throws Exception { "org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl"); brokerEnvs.put("loadBalancerLoadSheddingStrategy", "org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder"); + brokerEnvs.put("loadManagerServiceUnitStateTableViewClassName", + serviceUnitStateTableViewClassName); brokerEnvs.put("forceDeleteNamespaceAllowed", "true"); brokerEnvs.put("loadBalancerDebugModeEnabled", "true"); brokerEnvs.put("PULSAR_MEM", "-Xmx512M"); @@ -226,17 +244,17 @@ public void testSplitBundleAdminApi() throws Exception { long mid = bundleRanges.get(0) + (bundleRanges.get(1) - bundleRanges.get(0)) / 2; Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollDelay(100, TimeUnit.MILLISECONDS) .untilAsserted( - () -> { - BundlesData bundlesData = admin.namespaces().getBundles(DEFAULT_NAMESPACE); - assertEquals(bundlesData.getNumBundles(), numBundles + 1); - String lowBundle = String.format("0x%08x", bundleRanges.get(0)); - String midBundle = String.format("0x%08x", mid); - String highBundle = String.format("0x%08x", bundleRanges.get(1)); - assertTrue(bundlesData.getBoundaries().contains(lowBundle)); - assertTrue(bundlesData.getBoundaries().contains(midBundle)); - assertTrue(bundlesData.getBoundaries().contains(highBundle)); - } - ); + () -> { + BundlesData bundlesData = admin.namespaces().getBundles(DEFAULT_NAMESPACE); + assertEquals(bundlesData.getNumBundles(), numBundles + 1); + String lowBundle = String.format("0x%08x", bundleRanges.get(0)); + String midBundle = String.format("0x%08x", mid); + String highBundle = String.format("0x%08x", bundleRanges.get(1)); + assertTrue(bundlesData.getBoundaries().contains(lowBundle)); + assertTrue(bundlesData.getBoundaries().contains(midBundle)); + assertTrue(bundlesData.getBoundaries().contains(highBundle)); + } + ); // Test split bundle with invalid bundle range. @@ -393,10 +411,10 @@ public void testIsolationPolicy() throws Exception { } Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( - () -> { - List activeBrokers = admin.brokers().getActiveBrokersAsync().get(5, TimeUnit.SECONDS); - assertEquals(activeBrokers.size(), 1); - } + () -> { + List activeBrokers = admin.brokers().getActiveBrokersAsync().get(5, TimeUnit.SECONDS); + assertEquals(activeBrokers.size(), 1); + } ); Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted(