From d9e101a2b4d987210b0bb367a5c6872f1819900b Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Thu, 12 Sep 2024 14:34:24 -0700 Subject: [PATCH] [improve][broker] Add ServiceUnitStateTableView (ExtensibleLoadManagerImpl only) --- conf/broker.conf | 3 + .../pulsar/broker/ServiceConfiguration.java | 8 + .../extensions/ExtensibleLoadManagerImpl.java | 53 +- .../extensions/channel/ServiceUnitState.java | 24 +- .../channel/ServiceUnitStateChannel.java | 126 +-- .../channel/ServiceUnitStateChannelImpl.java | 365 ++++--- .../channel/ServiceUnitStateData.java | 15 + ...ServiceUnitStateDataConflictResolver.java} | 22 +- ...ceUnitStateMetadataStoreTableViewImpl.java | 155 +++ .../channel/ServiceUnitStateTableView.java | 113 ++ .../ServiceUnitStateTableViewBase.java | 92 ++ .../ServiceUnitStateTableViewImpl.java | 177 ++++ .../extensions/store/LoadDataStore.java | 6 + .../store/TableViewLoadDataStoreImpl.java | 144 ++- .../service/persistent/PersistentTopic.java | 8 +- .../ExtensibleLoadManagerImplBaseTest.java | 18 +- .../ExtensibleLoadManagerImplTest.java | 471 ++++----- ...anagerImplWithAdvertisedListenersTest.java | 7 +- ...gerImplWithTransactionCoordinatorTest.java | 6 +- .../channel/ServiceUnitStateChannelTest.java | 976 ++++++++++-------- ...iceUnitStateDataConflictResolverTest.java} | 32 +- .../channel/ServiceUnitStateTest.java | 176 +++- .../extensions/store/LoadDataStoreTest.java | 59 +- .../BrokerServiceAutoTopicCreationTest.java | 6 +- .../broker/service/BrokerServiceTest.java | 4 +- .../ServiceUnitStateCompactionTest.java | 29 +- .../compaction/StrategicCompactionTest.java | 2 +- .../metadata/api/MetadataStoreTableView.java | 126 +++ .../impl/MetadataStoreTableViewImpl.java | 313 ++++++ .../metadata/tableview/impl/package-info.java | 19 + .../metadata/MetadataStoreTableViewTest.java | 499 +++++++++ .../ExtensibleLoadManagerTest.java | 48 +- 32 files changed, 2986 insertions(+), 1116 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/{ServiceUnitStateCompactionStrategy.java => ServiceUnitStateDataConflictResolver.java} (82%) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java rename pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/{ServiceUnitStateCompactionStrategyTest.java => ServiceUnitStateDataConflictResolverTest.java} (88%) create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java create mode 100644 pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java diff --git a/conf/broker.conf b/conf/broker.conf index ed59e5c4566958..fa1033db6aa520 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1544,6 +1544,9 @@ 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 + ### --- Replication --- ### 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 60f37f52b6b8c6..41f83c3d9c0fdd 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 @@ -2874,6 +2874,14 @@ 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"; + /**** --- Replication. --- ****/ @FieldContext( category = CATEGORY_REPLICATION, 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 8e34f2f697fb18..ca01c8242277e2 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,16 @@ 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.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -209,46 +204,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 @@ -799,8 +766,8 @@ 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(); } catch (IOException ex) { 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 42ef55593ae1a0..b823a8277d3766 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 9be76e1b0f44d7..f8b95693f1ca57 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. @@ -51,92 +52,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); @@ -144,47 +92,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); @@ -195,18 +135,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. */ @@ -218,7 +164,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 1063f8124ece8d..a9ba182f403b6e 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); @@ -265,6 +258,23 @@ public void cleanOwnerships() { doCleanup(brokerId); } + + 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()); @@ -284,55 +294,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()); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, pulsar.getConfiguration().getClusterName(), + pulsar.getConfiguration().getDefaultNumberOfNamespaceBundles()); - ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); + tableview = createServiceUnitStateTableView(); + tableview.start(pulsar, this::handleEvent, this::handleExisting); - 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", - 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."); } @@ -373,23 +345,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) { @@ -427,6 +391,7 @@ private boolean debug() { return ExtensibleLoadManagerImpl.debug(config, log); } + @Override public CompletableFuture> getChannelOwnerAsync() { if (!validateChannelState(LeaderElectionServiceStarted, true)) { return CompletableFuture.failedFuture( @@ -437,6 +402,7 @@ public CompletableFuture> getChannelOwnerAsync() { .thenApply(leader -> leader.map(LeaderBroker::getBrokerId)); } + @Override public CompletableFuture isChannelOwnerAsync() { return getChannelOwnerAsync().thenApply(owner -> { if (owner.isPresent()) { @@ -448,19 +414,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()); } @@ -469,12 +430,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); } @@ -507,13 +469,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) { @@ -544,18 +515,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 -> { @@ -573,13 +545,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(); } @@ -602,22 +575,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())); @@ -626,7 +600,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); @@ -636,11 +611,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()); } @@ -692,12 +668,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) { @@ -710,7 +691,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; @@ -909,26 +890,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) { @@ -938,11 +917,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); @@ -962,7 +942,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()) @@ -1068,7 +1048,6 @@ private CompletableFuture splitServiceUnit(String serviceUnit, ServiceUnit } - @VisibleForTesting protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, NamespaceBundleFactory bundleFactory, @@ -1088,7 +1067,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", @@ -1104,7 +1083,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 { @@ -1151,45 +1130,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( @@ -1210,7 +1187,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(); @@ -1219,7 +1201,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); @@ -1258,8 +1263,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(); @@ -1390,7 +1400,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) { try { if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) .isEmpty()) { @@ -1424,7 +1434,7 @@ private synchronized void doCleanup(String broker) { } 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); } @@ -1448,7 +1458,7 @@ private synchronized void doCleanup(String broker) { } 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); } @@ -1481,8 +1491,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; } @@ -1572,7 +1587,7 @@ protected void monitorOwnerships(List brokers) { } 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); } @@ -1633,10 +1648,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; @@ -1782,7 +1795,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 307d3a4acb1759..e85134e6116325 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 6a98b79be81d01..b1dbb6fac8709e 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 00000000000000..ad58c8b35ec674 --- /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()) { + 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 00000000000000..5ac57fe5c19c61 --- /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 00000000000000..b690ef101e1687 --- /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 00000000000000..8dfaddcdabca11 --- /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/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java index a7deeeaad8a5cf..8096d1908b928f 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 e9289d3ccdac2f..b01a5ba9000966 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; @@ -114,17 +147,27 @@ public synchronized void closeTableView() throws IOException { } @Override - public synchronized void start() throws LoadDataStoreException { + public 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 f90e10ee64e656..e11d7c84526a3c 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; @@ -95,8 +96,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; @@ -251,8 +251,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 e9fafa9c30317d..1ccc927e378c37 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 @@ -30,6 +30,8 @@ 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; @@ -42,6 +44,7 @@ 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 +70,19 @@ public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServ protected LookupService lookupService; - protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { + protected String serviceUnitStateTableViewClassName; + + @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 +96,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)); 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 51966f420bf250..992b4b5ae0e912 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,7 @@ 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.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -102,6 +100,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 +137,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 +148,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()); @@ -417,7 +418,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 +458,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") @@ -544,8 +548,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 +565,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 +584,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 +600,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 @@ -687,7 +694,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) { @@ -802,10 +809,14 @@ public static void testOptimizeUnloadDisable(TopicDomain topicDomain, return sendCount.incrementAndGet() == 10; }); - assertTrue(producer.isConnected()); - assertTrue(consumer.isConnected()); - assertTrue(unloadNamespaceBundle.isDone()); - verify(lookup, times(2)).getBroker(topicName); + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertTrue(producer.isConnected()); + assertTrue(consumer.isConnected()); + assertTrue(unloadNamespaceBundle.isDone()); + verify(lookup, times(2)).getBroker(topicName); + }); } protected static LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { @@ -1043,52 +1054,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 +1177,49 @@ public void testDeployAndRollbackLoadManager() throws Exception { 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()); - 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()); - } - } - // 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"); } + } } private void assertLookupHeartbeatOwner(PulsarService pulsar, @@ -1272,12 +1286,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 +1301,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 +1320,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 +1336,6 @@ public void testRoleChangeIdempotency() throws Exception { FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); - - primaryLoadManager.playLeader(); secondaryLoadManager.playFollower(); verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); @@ -1334,6 +1343,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 +1366,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 +1386,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 +1408,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 +1420,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 +1440,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 +1643,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 +1654,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 +1765,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 +1807,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 +1823,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 +1852,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 +1865,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 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 bec7d4d78fe7e3..848660be8f3a5f 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 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 ed99b502b7e295..1d3f02f4e717d7 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 e569f0d32d573b..92cdf61f442693 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 049da191a80abb..d336e8918ec5e9 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 620266aee46a18..0a5f012ad40a7c 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 d25cba2bd1bdd2..3267e67ad2c3e5 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 ea5365bcf4b2c0..a4460187d2377b 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 2f27d5917f025f..5398b5aa57b8bb 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 9140216810826b..a834fa1fde1e35 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 d1ff46cbc02d5a..8f67e412267a0c 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/MetadataStoreTableView.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java new file mode 100644 index 00000000000000..a2238740ceafb9 --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java @@ -0,0 +1,126 @@ +/* + * 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.Collection; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; + +/** + * 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; + + /** + * Reads whether a specific key exists in the local tableview. + * + * @param key the key to check + * @return true if exists. Otherwise, false. + */ + boolean exists(String key); + + /** + * 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 size of the items in the local tableview. + * @return size + */ + int size(); + + /** + * Reads whether the local tableview is empty or not. + * @return true if empty. Otherwise, false + */ + boolean isEmpty(); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); + + /** + * Returns the key set of the items in the local tableview. + * @return key set + */ + Set keySet(); + + /** + * Returns the values of the items in the local tableview. + * @return values + */ + Collection values(); + + /** + * Runs the action for each item in the local tableview. + */ + void forEach(BiConsumer action); +} + 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 00000000000000..4ea5ba94caad5f --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java @@ -0,0 +1,313 @@ +/* + * 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.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.concurrent.locks.ReentrantLock; +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.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 MAX_CONCURRENT_METADATA_OPS_DURING_FILL = 50; + private static final long CACHE_REFRESH_FREQUENCY_IN_MILLIS = 600_000; + private static final long CACHE_EXPIRE_FREQUENCY_IN_MILLIS = 3600_000; + private final ReentrantLock listenersMutex; + 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.listenersMutex = new ReentrantLock(); + 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(CACHE_EXPIRE_FREQUENCY_IN_MILLIS) + .refreshAfterWriteMillis(CACHE_REFRESH_FREQUENCY_IN_MILLIS) + .build()); + store.registerListener(this::handleNotification); + } + + public void start() throws MetadataStoreException { + fill(); + } + + + 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 CompletableFuture doHandleNotification(String path) { + if (listenPathValidator != null && !listenPathValidator.test(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path).thenAccept(valOpt -> { + var val = valOpt.orElse(null); + String key = getKey(path); + + 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); + } + } + } + + }).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 (listenPathValidator != null && !listenPathValidator.test(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path) + .thenAccept(valOpt -> { + valOpt.ifPresent(val -> { + String key = getKey(path); + try { + listenersMutex.lock(); + 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; + } + } + } finally { + listenersMutex.unlock(); + } + }); + }); + } + + private void fill() throws MetadataStoreException { + 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()) { + 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(timeoutInMillis * size, 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 00000000000000..2c47770610b059 --- /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/MetadataStoreTableViewTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java new file mode 100644 index 00000000000000..5a2ea32890dbd3 --- /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 ee7497010adfcc..759e689b24d0f7 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(