From c94a1a2760edaa43c1d0149a9f0daa2ecab26c09 Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Tue, 14 Feb 2023 10:16:32 -0800 Subject: [PATCH 1/7] [improve][broker] PIP-192 Added Disabled and Init states in ServiceUnitState --- .../pulsar/broker/ServiceConfiguration.java | 17 + .../ExtensibleLoadManagerWrapper.java | 4 + .../extensions/channel/ServiceUnitState.java | 53 +-- .../channel/ServiceUnitStateChannelImpl.java | 237 +++++++++---- .../ServiceUnitStateCompactionStrategy.java | 80 +++-- .../channel/ServiceUnitStateData.java | 5 +- .../StrategicTwoPhaseCompactor.java | 39 ++- .../ExtensibleLoadManagerImplTest.java | 166 +++++----- .../channel/ServiceUnitStateChannelTest.java | 224 ++++++++++--- ...erviceUnitStateCompactionStrategyTest.java | 82 +++-- .../channel/ServiceUnitStateDataTest.java | 7 +- .../channel/ServiceUnitStateTest.java | 40 ++- .../ServiceUnitStateCompactionTest.java | 311 ++++++++++++------ 13 files changed, 869 insertions(+), 396 deletions(-) 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 106410d855e22..a846cc002d39f 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 @@ -2450,6 +2450,7 @@ The delayed message index bucket time step(in seconds) in per bucket snapshot se ) private long namespaceBundleUnloadingTimeoutMs = 60000; + /**** --- Load Balancer Extension. --- ****/ @FieldContext( category = CATEGORY_LOAD_BALANCER, dynamic = true, @@ -2525,6 +2526,22 @@ The delayed message index bucket time step(in seconds) in per bucket snapshot se ) private double loadBalancerBundleLoadReportPercentage = 10; + @FieldContext( + category = CATEGORY_LOAD_BALANCER, + doc = "After this delay, the service-unit state channel tombstones any service units (e.g., bundles) " + + "in semi-terminal states. For example, after splits, parent bundles will be `Disabled`, " + + "and then after this delay, the parent bundles' state will be removed(tombstoned) " + + "in the service-unit state channel. " + + "Pulsar does not immediately remove such semi-terminal states " + + "to avoid unnecessary system confusion, " + + "as the bundles in the `Tombstoned` state might temporarily look available to reassign. " + + "Rarely, one could lower this delay in order to aggressively clean " + + "the service-unit state channel when there are a large number of bundles. " + + "minimum value = 30 secs" + + "(only used in load balancer extension logics)" + ) + private long loadBalancerServiceUnitStateCleanUpDelayTimeInSeconds = 604800; + /**** --- Replication. --- ****/ @FieldContext( category = CATEGORY_REPLICATION, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index 48fc4bb7ff4f0..1eabbe620e213 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -142,4 +142,8 @@ public void doNamespaceBundleSplit() { throw new UnsupportedOperationException(); } + public ExtensibleLoadManagerImpl get() { + return loadManager; + } + } 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 3225c0ba7bbc7..5043aa84936ec 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 @@ -24,38 +24,13 @@ /** * Defines the possible states for service units. * - * The following diagram defines the valid state changes - * - * ┌───────────┐ - * ┌──────────┤ released │◄────────┐ - * │own └───────────┘ │release - * │ │ - * │ │ - * ▼ │ - * ┌────────┐ assign(transfer) ┌─────┴────┐ - * │ ├───────────────────►│ │ - * │ owned │ │ assigned │ - * │ │◄───────────────────┤ │ - * └──┬─────┤ own └──────────┘ - * │ ▲ │ ▲ - * │ │ │ │ - * │ │ └──────────────┐ │ - * │ │ │ │ - * │ │ unload │ │ assign(assignment) - * split │ │ │ │ - * │ │ │ │ - * │ │ create(child) │ │ - * │ │ │ │ - * ▼ │ │ │ - * ┌─────┴─────┐ └─────►┌───┴──────┐ - * │ │ │ │ - * │ splitting ├────────────────► │ free │ - * │ │ discard(parent)│ │ - * └───────────┘ └──────────┘ + * Refer to Service Unit State Channel in https://github.com/apache/pulsar/issues/16691 for additional details. */ public enum ServiceUnitState { - Free, // not owned by any broker (terminal state) + Init, // initializing the state. no previous state(terminal state) + + Free, // not owned by any broker (semi-terminal state) Owned, // owned by a broker (terminal state) @@ -63,16 +38,20 @@ public enum ServiceUnitState { Released, // the source broker's ownership has been released (e.g. the topic connections are closed) - Splitting; // the service unit(e.g. bundle) is in the process of splitting. + Splitting, // the service unit(e.g. bundle) is in the process of splitting. + + Disabled; // disabled in the system (semi-terminal state) private static Map> validTransitions = Map.of( - // (Free -> Released | Splitting) transitions are required - // when the topic is compacted in the middle of transfer or split. - Free, Set.of(Owned, Assigned, Released, Splitting), - Owned, Set.of(Assigned, Splitting, Free), - Assigned, Set.of(Owned, Released, Free), - Released, Set.of(Owned, Free), - Splitting, Set.of(Free) + // (Init -> all states) transitions are required + // when the topic is compacted in the middle of assign, transfer or split. + Init, Set.of(Free, Owned, Assigned, Released, Splitting, Disabled, Init), + Free, Set.of(Assigned, Init), + Owned, Set.of(Assigned, Splitting, Free, Init), + Assigned, Set.of(Owned, Released, Init), + Released, Set.of(Owned, Init), + Splitting, Set.of(Disabled, Init), + Disabled, Set.of(Init) ); public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to) { 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 d10138bda6805..051e19b61ea4f 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 @@ -20,8 +20,11 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; @@ -61,7 +64,10 @@ import org.apache.commons.lang3.mutable.MutableInt; 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.extensions.BrokerRegistry; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; @@ -92,27 +98,31 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { TopicDomain.persistent.value(), NamespaceName.SYSTEM_NAMESPACE, "loadbalancer-service-unit-state").toString(); - - // TODO: define StateCompactionStrategy private static final long MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS = 30 * 1000; // 30sec + + private static final long OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS = 60; 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 final PulsarService pulsar; + private final ServiceConfiguration config; private final Schema schema; private final ConcurrentOpenHashMap> getOwnerRequests; private final String lookupServiceAddress; - // TODO: define BrokerRegistry private final ConcurrentOpenHashMap> cleanupJobs; private final LeaderElectionService leaderElectionService; + private BrokerRegistry brokerRegistry; private TableView tableview; private Producer producer; - private ScheduledFuture cleanupTasks; + private ScheduledFuture monitorTask; private SessionEvent lastMetadataSessionEvent = SessionReestablished; private long lastMetadataSessionEventTimestamp = 0; private long inFlightStateWaitingTimeInMillis; + + private long ownershipMonitorDelayTimeInSecs; + private long semiTerminalStateWaitingTimeInMillis; private long maxCleanupDelayTimeInSecs; private long minCleanupDelayTimeInSecs; // cleanup metrics @@ -125,6 +135,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private long totalCleanupCancelledCnt = 0; private volatile ChannelState channelState; + private AtomicLong totalBundleSplitRetryCount = new AtomicLong(); + public enum EventType { Assign, Split, @@ -135,30 +147,18 @@ public enum EventType { @Getter @AllArgsConstructor public static class Counters { - private AtomicLong total; - private AtomicLong failure; + private final AtomicLong total; + private final AtomicLong failure; + public Counters(){ + total = new AtomicLong(); + failure = new AtomicLong(); + } } // operation metrics - final Map ownerLookUpCounters = Map.of( - Owned, new AtomicLong(), - Assigned, new AtomicLong(), - Released, new AtomicLong(), - Splitting, new AtomicLong(), - Free, new AtomicLong() - ); - final Map eventCounters = Map.of( - Assign, new Counters(new AtomicLong(), new AtomicLong()), - Split, new Counters(new AtomicLong(), new AtomicLong()), - Unload, new Counters(new AtomicLong(), new AtomicLong()) - ); - final Map handlerCounters = Map.of( - Owned, new Counters(new AtomicLong(), new AtomicLong()), - Assigned, new Counters(new AtomicLong(), new AtomicLong()), - Released, new Counters(new AtomicLong(), new AtomicLong()), - Splitting, new Counters(new AtomicLong(), new AtomicLong()), - Free, new Counters(new AtomicLong(), new AtomicLong()) - ); + final Map ownerLookUpCounters; + final Map eventCounters; + final Map handlerCounters; enum ChannelState { Closed(0), @@ -180,25 +180,61 @@ enum MetadataState { public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.pulsar = pulsar; + this.config = pulsar.getConfig(); this.lookupServiceAddress = pulsar.getLookupServiceAddress(); this.schema = Schema.JSON(ServiceUnitStateData.class); this.getOwnerRequests = ConcurrentOpenHashMap.>newBuilder().build(); this.cleanupJobs = ConcurrentOpenHashMap.>newBuilder().build(); + this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateCleanUpDelayTimeInSeconds() + * 1000; this.inFlightStateWaitingTimeInMillis = MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS; + this.ownershipMonitorDelayTimeInSecs = OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS; + if (semiTerminalStateWaitingTimeInMillis < inFlightStateWaitingTimeInMillis) { + throw new IllegalArgumentException( + "Invalid Config: loadBalancerServiceUnitStateCleanUpDelayTimeInSeconds < " + + (MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS / 1000) + " secs"); + } this.maxCleanupDelayTimeInSecs = MAX_CLEAN_UP_DELAY_TIME_IN_SECS; this.minCleanupDelayTimeInSecs = MIN_CLEAN_UP_DELAY_TIME_IN_SECS; this.leaderElectionService = new LeaderElectionService( pulsar.getCoordinationService(), pulsar.getSafeWebServiceAddress(), state -> { if (state == LeaderElectionState.Leading) { - log.debug("This broker:{} is the leader now.", lookupServiceAddress); - // TODO: schedule monitorOwnerships by brokerRegistry + log.info("This broker:{} is the leader now.", lookupServiceAddress); + this.monitorTask = this.pulsar.getLoadManagerExecutor() + .scheduleWithFixedDelay(() -> { + try { + monitorOwnerships(brokerRegistry.getAvailableBrokersAsync() + .get(inFlightStateWaitingTimeInMillis, MILLISECONDS)); + } catch (Exception e) { + log.info("Failed to monitor the ownerships. will retry..", e); + } + }, + ownershipMonitorDelayTimeInSecs, ownershipMonitorDelayTimeInSecs, SECONDS); } else { - log.debug("This broker:{} is a follower now.", lookupServiceAddress); - // TODO: cancel scheduled monitorOwnerships if any + log.info("This broker:{} is a follower now.", lookupServiceAddress); + if (monitorTask != null) { + monitorTask.cancel(false); + monitorTask = null; + log.info("This previous leader broker:{} stopped the channel clean-up monitor", + lookupServiceAddress); + } } }); + Map tmpOwnerLookUpCounters = new HashMap<>(); + Map tmpHandlerCounters = new HashMap<>(); + Map tmpEventCounters = new HashMap<>(); + for (var state : ServiceUnitState.values()) { + tmpOwnerLookUpCounters.put(state, new AtomicLong()); + tmpHandlerCounters.put(state, new Counters()); + } + for (var event : EventType.values()) { + tmpEventCounters.put(event, new Counters()); + } + ownerLookUpCounters = Map.copyOf(tmpOwnerLookUpCounters); + handlerCounters = Map.copyOf(tmpHandlerCounters); + eventCounters = Map.copyOf(tmpEventCounters); this.channelState = Constructed; } @@ -207,14 +243,29 @@ public synchronized void start() throws PulsarServerException { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } + boolean debug = debug(); try { + this.brokerRegistry = getBrokerRegistry(); + this.brokerRegistry.addListener((broker, type) -> { + if (type == NotificationType.Deleted) { + log.info("BrokerRegistry detected the broker:{} registry has been deleted.", broker); + handleBrokerDeletionEvent(broker); + } else if (type == NotificationType.Created) { + log.info("BrokerRegistry detected the broker:{} registry has been created.", broker); + handleBrokerCreationEvent(broker); + } + }); leaderElectionService.start(); this.channelState = LeaderElectionServiceStarted; - log.debug("Successfully started the channel leader election service."); + if (debug) { + log.info("Successfully started the channel leader election service."); + } if (producer != null) { producer.close(); - log.debug("Closed the channel producer."); + if (debug) { + log.info("Closed the channel producer."); + } } producer = pulsar.getClient().newProducer(schema) .enableBatching(true) @@ -223,11 +274,15 @@ public synchronized void start() throws PulsarServerException { .topic(TOPIC) .create(); - log.debug("Successfully started the channel producer."); + if (debug) { + log.info("Successfully started the channel producer."); + } if (tableview != null) { tableview.close(); - log.debug("Closed the channel tableview."); + if (debug) { + log.info("Closed the channel tableview."); + } } tableview = pulsar.getClient().newTableViewBuilder(schema) .topic(TOPIC) @@ -236,10 +291,13 @@ public synchronized void start() throws PulsarServerException { ServiceUnitStateCompactionStrategy.class.getName())) .create(); tableview.listen((key, value) -> handle(key, value)); - log.debug("Successfully started the channel tableview."); - + if (debug) { + log.info("Successfully started the channel tableview."); + } pulsar.getLocalMetadataStore().registerSessionListener(this::handleMetadataSessionEvent); - log.debug("Successfully registered the handleMetadataSessionEvent"); + if (debug) { + log.info("Successfully registered the handleMetadataSessionEvent"); + } channelState = Started; log.info("Successfully started the channel."); @@ -250,16 +308,27 @@ public synchronized void start() throws PulsarServerException { } } + @VisibleForTesting + protected BrokerRegistry getBrokerRegistry() { + return ((ExtensibleLoadManagerWrapper) pulsar.getLoadManager().get()) + .get().getBrokerRegistry(); + } + public synchronized void close() throws PulsarServerException { channelState = Closed; + boolean debug = debug(); try { leaderElectionService.close(); - log.debug("Successfully closed the channel leader election service."); + if (debug) { + log.info("Successfully closed the channel leader election service."); + } if (tableview != null) { tableview.close(); tableview = null; - log.debug("Successfully closed the channel tableview."); + if (debug) { + log.info("Successfully closed the channel tableview."); + } } if (producer != null) { @@ -268,11 +337,13 @@ public synchronized void close() throws PulsarServerException { log.info("Successfully closed the channel producer."); } - // TODO: clean brokerRegistry + if (brokerRegistry != null) { + brokerRegistry = null; + } - if (cleanupTasks != null) { - cleanupTasks.cancel(true); - cleanupTasks = null; + if (monitorTask != null) { + monitorTask.cancel(true); + monitorTask = null; log.info("Successfully cancelled the cleanup tasks"); } @@ -294,7 +365,7 @@ private boolean validateChannelState(ChannelState targetState, boolean checkLowe } private boolean debug() { - return pulsar.getConfiguration().isLoadBalancerDebugModeEnabled() || log.isDebugEnabled(); + return config.isLoadBalancerDebugModeEnabled() || log.isDebugEnabled(); } public CompletableFuture> getChannelOwnerAsync() { @@ -348,7 +419,7 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } ServiceUnitStateData data = tableview.get(serviceUnit); - ServiceUnitState state = data == null ? Free : data.state(); + ServiceUnitState state = data == null ? Init : data.state(); ownerLookUpCounters.get(state).incrementAndGet(); switch (state) { case Owned, Splitting -> { @@ -357,9 +428,12 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { case Assigned, Released -> { return deferGetOwnerRequest(serviceUnit).thenApply(Optional::of); } - case Free -> { + case Free, Init -> { return CompletableFuture.completedFuture(Optional.empty()); } + case Disabled -> { + return CompletableFuture.failedFuture(new IllegalArgumentException(serviceUnit + " is disabled")); + } default -> { String errorMsg = String.format("Failed to process service unit state data: %s when get owner.", data); log.error(errorMsg); @@ -391,11 +465,11 @@ public CompletableFuture publishUnloadEventAsync(Unload unload) { String serviceUnit = unload.serviceUnit(); CompletableFuture future; if (isTransferCommand(unload)) { - ServiceUnitStateData next = new ServiceUnitStateData(Assigned, - unload.destBroker().get(), unload.sourceBroker()); - future = pubAsync(serviceUnit, next); + future = pubAsync(serviceUnit, new ServiceUnitStateData( + Assigned, unload.destBroker().get(), unload.sourceBroker())); } else { - future = tombstoneAsync(serviceUnit); + future = pubAsync(serviceUnit, new ServiceUnitStateData( + Free, unload.sourceBroker())); } return future.whenComplete((__, ex) -> { @@ -424,14 +498,14 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { lookupServiceAddress, serviceUnit, data, totalHandledRequests); } - ServiceUnitState state = data == null ? Free : data.state(); + ServiceUnitState state = data == null ? Init : data.state(); try { switch (state) { case Owned -> handleOwnEvent(serviceUnit, data); case Assigned -> handleAssignEvent(serviceUnit, data); case Released -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); - case Free -> handleFreeEvent(serviceUnit); + case Free, Disabled, Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } } catch (Throwable e){ @@ -453,7 +527,7 @@ private static boolean isTransferCommand(Unload data) { } private static String getLogEventTag(ServiceUnitStateData data) { - return data == null ? "Free" : + return data == null ? Init.toString() : isTransferCommand(data) ? "Transfer:" + data.state() : data.state().toString(); } @@ -466,7 +540,7 @@ private AtomicLong getHandlerFailureCounter(ServiceUnitStateData data) { } private AtomicLong getHandlerCounter(ServiceUnitStateData data, boolean total) { - var state = data == null ? Free : data.state(); + var state = data == null ? Init : data.state(); var counter = total ? handlerCounters.get(state).getTotal() : handlerCounters.get(state).getFailure(); if (counter == null) { @@ -538,7 +612,7 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { } } - private void handleFreeEvent(String serviceUnit) { + private void handleInitEvent(String serviceUnit) { closeServiceUnit(serviceUnit) .thenAccept(__ -> { var request = getOwnerRequests.remove(serviceUnit); @@ -658,6 +732,10 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, CompletableFuture completionFuture) { CompletableFuture> updateFuture = new CompletableFuture<>(); + if (counter.get() > 0) { + totalBundleSplitRetryCount.incrementAndGet(); + } + pulsar.getNamespaceService().getSplitBoundary(bundle, null).thenAccept(splitBundlesPair -> { // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper. if (splitBundlesPair == null) { @@ -702,8 +780,8 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, }); updateFuture.thenAccept(r -> { - // Free the old bundle - tombstoneAsync(serviceUnit).thenRun(() -> { + // Disable the old bundle + pubAsync(serviceUnit, new ServiceUnitStateData(Disabled, data.broker())).thenRun(() -> { // Update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(bundle); // TODO: Update the load data immediately if needed. @@ -723,6 +801,8 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, Throwable throwable = FutureUtil.unwrapCompletionException(ex); if ((throwable instanceof MetadataStoreException.BadVersionException) && (counter.incrementAndGet() < NamespaceService.BUNDLE_SPLIT_RETRY_LIMIT)) { + 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, bundle, serviceUnit, data, counter, startTime, completionFuture), 100, MILLISECONDS); } else if (throwable instanceof IllegalArgumentException) { @@ -838,7 +918,7 @@ private void doCleanup(String broker) { log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " + "cleanupErrorCnt:{}.", serviceUnit, stateData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart); + totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); } }); serviceUnitTombstoneCnt++; @@ -871,8 +951,8 @@ private void doCleanup(String broker) { cleanupJobs.remove(broker); } - // TODO: integrate this monitor logic when broker registry is added - private void monitorOwnerships(List brokers) { + @VisibleForTesting + protected void monitorOwnerships(List brokers) { if (!isChannelOwner()) { log.warn("This broker is not the leader now. Skipping ownership monitor"); return; @@ -893,22 +973,37 @@ private void monitorOwnerships(List brokers) { String serviceUnit = etr.getKey(); ServiceUnitStateData stateData = etr.getValue(); String broker = stateData.broker(); + var state = stateData.state(); if (!activeBrokers.contains(broker)) { inactiveBrokers.add(stateData.broker()); - } else if (stateData.state() != Owned + } else if (state != Owned && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { - log.warn("Found long-running orphan(in-flight) serviceUnit:{}, stateData:{}", - serviceUnit, stateData); - - tombstoneAsync(serviceUnit).whenComplete((__, e) -> { - if (e != null) { - log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " - + "cleanupErrorCnt:{}.", - serviceUnit, stateData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart); + boolean tombstone = false; + if ((state == Free || state == Disabled)) { + if (now - stateData.timestamp() + > semiTerminalStateWaitingTimeInMillis) { + log.info("Found semi-terminal states(free or disabled) to clean" + + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); + tombstone = true; } - }); - serviceUnitTombstoneCnt++; + } else { + log.warn("Found long-running orphan(in-flight) serviceUnit:{}, stateData:{}", + serviceUnit, stateData); + tombstone = true; + } + + + if (tombstone) { + tombstoneAsync(serviceUnit).whenComplete((__, e) -> { + if (e != null) { + log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " + + "cleanupErrorCnt:{}.", + serviceUnit, stateData, + totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); + } + }); + serviceUnitTombstoneCnt++; + } } } 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/ServiceUnitStateCompactionStrategy.java index 2b21f830dda92..19f928b23bae1 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/ServiceUnitStateCompactionStrategy.java @@ -18,13 +18,11 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.channel; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +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.Init; import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.topics.TopicCompactionStrategy; @@ -50,40 +48,56 @@ public void checkBrokers(boolean check) { @Override public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to) { - ServiceUnitState prevState = from == null ? Free : from.state(); - ServiceUnitState state = to == null ? Free : to.state(); + ServiceUnitState prevState = from == null ? Init : from.state(); + ServiceUnitState state = to == null ? Init : to.state(); if (!ServiceUnitState.isValidTransition(prevState, state)) { return true; } if (checkBrokers) { - if (prevState == Free && (state == Assigned || state == Owned)) { - // Free -> Assigned || Owned broker check - return StringUtils.isBlank(to.broker()); - } else if (prevState == Owned && state == Assigned) { - // Owned -> Assigned(transfer) broker check - return !StringUtils.equals(from.broker(), to.sourceBroker()) - || StringUtils.isBlank(to.broker()) - || StringUtils.equals(from.broker(), to.broker()); - } else if (prevState == Assigned && state == Released) { - // Assigned -> Released(transfer) broker check - return !StringUtils.equals(from.broker(), to.broker()) - || !StringUtils.equals(from.sourceBroker(), to.sourceBroker()); - } else if (prevState == Released && state == Owned) { - // Released -> Owned(transfer) broker check - return !StringUtils.equals(from.broker(), to.broker()) - || !StringUtils.equals(from.sourceBroker(), to.sourceBroker()); - } else if (prevState == Assigned && state == Owned) { - // Assigned -> Owned broker check - return !StringUtils.equals(from.broker(), to.broker()) - || !StringUtils.equals(from.sourceBroker(), to.sourceBroker()); - } else if (prevState == Owned && state == Splitting) { - // Owned -> Splitting broker check - return !StringUtils.equals(from.broker(), to.broker()); + switch (prevState) { + case Free: + switch (state) { + case Assigned: + return isNotBlank(to.sourceBroker()); + } + case Owned: + switch (state) { + case Assigned: + return invalidTransfer(from, to); + case Splitting: + case Free: + return !from.broker().equals(to.broker()); + } + case Assigned: + switch (state) { + case Released: + return isBlank(to.sourceBroker()) || notEquals(from, to); + case Owned: + return isNotBlank(to.sourceBroker()) || !to.broker().equals(from.broker()); + } + case Released: + switch (state) { + case Owned: + return notEquals(from, to); + } + case Splitting: + switch (state) { + case Disabled : + return notEquals(from, to); + } } } - return false; } -} + private boolean notEquals(ServiceUnitStateData from, ServiceUnitStateData to) { + return !from.broker().equals(to.broker()) + || !StringUtils.equals(from.sourceBroker(), to.sourceBroker()); + } + + private boolean invalidTransfer(ServiceUnitStateData from, ServiceUnitStateData to) { + return !from.broker().equals(to.sourceBroker()) + || from.broker().equals(to.broker()); + } +} \ No newline at end of file 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 cba459b7875f7..6a7bf97b46b88 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 @@ -20,6 +20,7 @@ import java.util.Objects; +import org.apache.commons.lang3.StringUtils; /** * Defines data for the service unit state changes. @@ -30,7 +31,9 @@ public record ServiceUnitStateData(ServiceUnitState state, String broker, String public ServiceUnitStateData { Objects.requireNonNull(state); - Objects.requireNonNull(broker); + if (StringUtils.isBlank(broker)) { + throw new IllegalArgumentException("Empty broker"); + } } public ServiceUnitStateData(ServiceUnitState state, String broker, String sourceBroker) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java index 9dc4ec649b62b..37b03e275d6bf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java @@ -38,7 +38,6 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.CompactionReaderImpl; @@ -63,6 +62,7 @@ public class StrategicTwoPhaseCompactor extends TwoPhaseCompactor { private static final Logger log = LoggerFactory.getLogger(StrategicTwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; + private static final int MAX_READER_RECONNECT_WAITING_TIME_IN_MILLIS = 20 * 1000; private final Duration phaseOneLoopReadTimeout; private final RawBatchMessageContainerImpl batchMessageContainer; @@ -110,7 +110,7 @@ CompletableFuture doCompaction(Reader reader, TopicCompactionStrate if (!(reader instanceof CompactionReaderImpl)) { return CompletableFuture.failedFuture( - new IllegalStateException("reader has to be DelayedAckReaderImpl")); + new IllegalStateException("reader has to be CompactionReaderImpl")); } return reader.hasMessageAvailableAsync() .thenCompose(available -> { @@ -284,9 +284,12 @@ private void phaseOneLoop(Reader reader, CompletableFuture void phaseOneLoop(Reader reader, CompletableFuture void waitForReconnection(Reader reader) { + long started = System.currentTimeMillis(); + + // initial sleep + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + while (!reader.isConnected()) { + long now = System.currentTimeMillis(); + if (now - started > MAX_READER_RECONNECT_WAITING_TIME_IN_MILLIS) { + String errorMsg = String.format( + "Reader has not been reconnected for %d secs. Stopping the compaction.", + MAX_READER_RECONNECT_WAITING_TIME_IN_MILLIS / 1000); + log.error(errorMsg); + throw new RuntimeException(errorMsg); + } + log.warn( + "Reader has not been reconnected after the cursor reset. elapsed :{} ms. Retrying " + + "soon.", now - started); + try { + Thread.sleep(100); + } catch (InterruptedException e) { + log.warn("The thread got interrupted while waiting. continuing", e); + } + } + } + private CompletableFuture phaseTwo(PhaseOneResult phaseOneResult, Reader reader, BookKeeper bk) { log.info("Completed phase one. Result:{}. ", phaseOneResult); Map metadata = 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 1ef4f660e4af3..19485b25fb4cc 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,14 +18,6 @@ */ package org.apache.pulsar.broker.loadbalance.extensions; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Assign; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Split; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload; 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; @@ -53,6 +45,7 @@ import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; +import java.util.LinkedHashMap; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -73,6 +66,7 @@ import org.apache.pulsar.broker.loadbalance.BrokerFilterException; import org.apache.pulsar.broker.loadbalance.LeaderBroker; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; +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; @@ -342,17 +336,19 @@ public void testGetMetrics() throws Exception { FieldUtils.writeDeclaredField(unloadCounter, "loadAvg", 1.5, true); FieldUtils.writeDeclaredField(unloadCounter, "loadStd", 0.3, true); FieldUtils.writeDeclaredField(unloadCounter, "breakdownCounters", Map.of( - Success, Map.of( - Overloaded, new MutableLong(1), - Underloaded, new MutableLong(2)), - Skip, Map.of( - Balanced, new MutableLong(3), - NoBundles, new MutableLong(4), - CoolDown, new MutableLong(5), - OutDatedData, new MutableLong(6), - NoLoadData, new MutableLong(7), - NoBrokers, new MutableLong(8), - Unknown, new MutableLong(9)), + Success, new LinkedHashMap<>() {{ + put(Overloaded, new MutableLong(1)); + put(Underloaded, new MutableLong(2)); + }}, + Skip, new LinkedHashMap<>() {{ + put(Balanced, new MutableLong(3)); + put(NoBundles, new MutableLong(4)); + put(CoolDown, new MutableLong(5)); + put(OutDatedData, new MutableLong(6)); + put(NoLoadData, new MutableLong(7)); + put(NoBrokers, new MutableLong(8)); + put(Unknown, new MutableLong(9)); + }}, Failure, Map.of( Unknown, new MutableLong(10)) ), true); @@ -363,19 +359,24 @@ Unknown, new MutableLong(10)) FieldUtils.readDeclaredField(primaryLoadManager, "splitMetrics", true); SplitCounter splitCounter = new SplitCounter(); FieldUtils.writeDeclaredField(splitCounter, "splitCount", 35l, true); - FieldUtils.writeDeclaredField(splitCounter, "breakdownCounters", Map.of( - SplitDecision.Label.Success, Map.of( - Topics, new MutableLong(1), - Sessions, new MutableLong(2), - MsgRate, new MutableLong(3), - Bandwidth, new MutableLong(4), - Admin, new MutableLong(5)), - SplitDecision.Label.Skip, Map.of( + FieldUtils.writeDeclaredField(splitCounter, "breakdownCounters", new LinkedHashMap<>() { + { + put(SplitDecision.Label.Success, new LinkedHashMap<>() { + { + put(Topics, new MutableLong(1)); + put(Sessions, new MutableLong(2)); + put(MsgRate, new MutableLong(3)); + put(Bandwidth, new MutableLong(4)); + put(Admin, new MutableLong(5)); + } + }); + put(SplitDecision.Label.Skip, Map.of( SplitDecision.Reason.Balanced, new MutableLong(6) - ), - SplitDecision.Label.Failure, Map.of( - SplitDecision.Reason.Unknown, new MutableLong(7)) - ), true); + )); + put(SplitDecision.Label.Failure, Map.of( + SplitDecision.Reason.Unknown, new MutableLong(7))); + } + }, true); splitMetrics.set(splitCounter.toMetrics(pulsar.getAdvertisedAddress())); } @@ -398,27 +399,30 @@ SplitDecision.Reason.Unknown, new MutableLong(7)) FieldUtils.writeDeclaredField(channel1, "totalCleanupScheduledCnt", 5, true); FieldUtils.writeDeclaredField(channel1, "totalCleanupIgnoredCnt", 6, true); FieldUtils.writeDeclaredField(channel1, "totalCleanupCancelledCnt", 7, true); - FieldUtils.writeDeclaredField(channel1, "ownerLookUpCounters", Map.of( - Owned, new AtomicLong(1), - Assigned, new AtomicLong(2), - Released, new AtomicLong(3), - Splitting, new AtomicLong(4), - Free, new AtomicLong(5) - ), true); - FieldUtils.writeDeclaredField(channel1, "eventCounters", Map.of( - Assign, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(1), new AtomicLong(2)), - Split, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(3), new AtomicLong(4)), - Unload, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(5), new AtomicLong(6)) - ), true); - - FieldUtils.writeDeclaredField(channel1, "handlerCounters", Map.of( - Owned, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(1), new AtomicLong(2)), - Assigned, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(3), new AtomicLong(4)), - Released, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(5), new AtomicLong(6)), - Splitting, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(7), new AtomicLong(8)), - Free, new ServiceUnitStateChannelImpl.Counters(new AtomicLong(9), new AtomicLong(10)) - ), true); - + Map ownerLookUpCounters = new LinkedHashMap<>(); + Map handlerCounters = new LinkedHashMap<>(); + Map eventCounters = + new LinkedHashMap<>(); + int i = 1; + int j = 0; + for (var state : ServiceUnitState.values()) { + ownerLookUpCounters.put(state, new AtomicLong(i)); + handlerCounters.put(state, + new ServiceUnitStateChannelImpl.Counters( + new AtomicLong(j + 1), new AtomicLong(j + 2))); + i++; + j += 2; + } + i = 0; + for (var type : ServiceUnitStateChannelImpl.EventType.values()) { + eventCounters.put(type, + new ServiceUnitStateChannelImpl.Counters( + new AtomicLong(i + 1), new AtomicLong(i + 2))); + i += 2; + } + FieldUtils.writeDeclaredField(channel1, "ownerLookUpCounters", ownerLookUpCounters, true); + FieldUtils.writeDeclaredField(channel1, "eventCounters", eventCounters, true); + FieldUtils.writeDeclaredField(channel1, "handlerCounters", handlerCounters, true); } var expected = Set.of( @@ -427,50 +431,56 @@ Free, new AtomicLong(5) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=OutDatedData, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=6}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoLoadData, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=7}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBrokers, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=8}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] - dimensions=[{broker=localhost, metric=bundlesSplit, reason=Bandwidth, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=4}] + dimensions=[{broker=localhost, metric=bundlesSplit, reason=Topics, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=1}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Sessions, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=2}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=MsgRate, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=3}] + dimensions=[{broker=localhost, metric=bundlesSplit, reason=Bandwidth, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Admin, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=5}] - dimensions=[{broker=localhost, metric=bundlesSplit, reason=Topics, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=1}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] + dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] - dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] - dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=5}] - dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=6}] - dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=3}] - dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=4}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Disabled}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=1}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=2}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=1}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=2}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] + dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=3}] + dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=4}] + dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=5}] + dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=6}] + dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=1}] + dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=2}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] + dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] + dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=4}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=6}] 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 327afa3cb8891..148ca982b11c7 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 @@ -19,7 +19,9 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; @@ -52,6 +54,7 @@ import static org.testng.AssertJUnit.assertNotNull; import java.lang.reflect.Field; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -71,6 +74,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; +import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistryImpl; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.namespace.NamespaceService; @@ -104,6 +108,8 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private String bundle2; private PulsarTestContext additionalPulsarTestContext; + private BrokerRegistryImpl registry; + @BeforeClass @Override protected void setup() throws Exception { @@ -117,11 +123,15 @@ protected void setup() throws Exception { admin.namespaces().createNamespace("public/default"); pulsar1 = pulsar; + registry = new BrokerRegistryImpl(pulsar); additionalPulsarTestContext = createAdditionalPulsarTestContext(getDefaultConf()); pulsar2 = additionalPulsarTestContext.getPulsarService(); - channel1 = spy(new ServiceUnitStateChannelImpl(pulsar1)); + + + channel1 = createChannel(pulsar1); channel1.start(); - channel2 = spy(new ServiceUnitStateChannelImpl(pulsar2)); + + channel2 = createChannel(pulsar2); channel2.start(); lookupServiceAddress1 = (String) FieldUtils.readDeclaredField(channel1, "lookupServiceAddress", true); @@ -137,6 +147,8 @@ protected void setup() throws Exception { protected void initTableViews() throws Exception { cleanTableView(channel1, bundle); cleanTableView(channel2, bundle); + cleanOwnershipMonitorCounters(channel1); + cleanOwnershipMonitorCounters(channel2); cleanOpsCounters(channel1); cleanOpsCounters(channel2); } @@ -187,7 +199,7 @@ public void channelOwnerTest() throws Exception { public void channelValidationTest() throws ExecutionException, InterruptedException, IllegalAccessException, PulsarServerException, TimeoutException { - var channel = new ServiceUnitStateChannelImpl(pulsar); + var channel = createChannel(pulsar); int errorCnt = validateChannelStart(channel); assertEquals(6, errorCnt); ExecutorService executor = Executors.newSingleThreadExecutor(); @@ -333,8 +345,8 @@ public void assignmentTest() assertEquals(getOwnerRequests1.size(), 0); assertEquals(getOwnerRequests2.size(), 0); - validateHandlerCounters(channel1, 1, 0, 1, 0, 0, 0, 0, 0, 0, 0); - validateHandlerCounters(channel2, 1, 0, 1, 0, 0, 0, 0, 0, 0, 0); + validateHandlerCounters(channel1, 1, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + validateHandlerCounters(channel2, 1, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0); validateEventCounters(channel1, 1, 0, 0, 0, 0, 0); validateEventCounters(channel2, 1, 0, 0, 0, 0, 0); } @@ -379,7 +391,7 @@ public void assignmentTestWhenOneAssignmentFails() } @Test(priority = 4) - public void unloadTest() + public void transferTest() throws ExecutionException, InterruptedException, TimeoutException, IllegalAccessException { var owner1 = channel1.getOwnerAsync(bundle); @@ -409,15 +421,16 @@ public void unloadTest() assertEquals(ownerAddr1, ownerAddr2); assertEquals(ownerAddr1, Optional.of(lookupServiceAddress2)); - validateHandlerCounters(channel1, 2, 0, 2, 0, 1, 0, 0, 0, 0, 0); - validateHandlerCounters(channel2, 2, 0, 2, 0, 1, 0, 0, 0, 0, 0); + validateHandlerCounters(channel1, 2, 0, 2, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0); + validateHandlerCounters(channel2, 2, 0, 2, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0); validateEventCounters(channel1, 1, 0, 0, 0, 1, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); } @Test(priority = 5) - public void unloadTestWhenDestBrokerFails() - throws ExecutionException, InterruptedException, IllegalAccessException { + public void transferTestWhenDestBrokerFails() + throws ExecutionException, InterruptedException, IllegalAccessException, PulsarServerException, + TimeoutException { var getOwnerRequests1 = getOwnerRequests(channel1); var getOwnerRequests2 = getOwnerRequests(channel2); @@ -450,8 +463,8 @@ public void unloadTestWhenDestBrokerFails() Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.of(lookupServiceAddress2)); channel1.publishUnloadEventAsync(unload); // channel1 is broken. the ownership transfer won't be complete. - waitUntilNewState(channel1, bundle); - waitUntilNewState(channel2, bundle); + waitUntilState(channel1, bundle); + waitUntilState(channel2, bundle); var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); @@ -461,7 +474,7 @@ public void unloadTestWhenDestBrokerFails() assertEquals(1, getOwnerRequests1.size()); assertEquals(1, getOwnerRequests2.size()); - // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. + // In 5 secs, the getOwnerAsync requests(lookup requests) should time out. Awaitility.await().atMost(5, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); Awaitility.await().atMost(5, TimeUnit.SECONDS) @@ -470,19 +483,26 @@ public void unloadTestWhenDestBrokerFails() assertEquals(0, getOwnerRequests1.size()); assertEquals(0, getOwnerRequests2.size()); + // recovered, check the monitor update state : Assigned -> Init + FieldUtils.writeDeclaredField(channel2, "producer", producer, true); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); - // TODO: retry lookups and assert that the monitor cleans up the stuck assignments - /* - owner1 = channel1.getOwnerAsync(bundle); - owner2 = channel2.getOwnerAsync(bundle); - assertFalse(channel1.getOwnerAsync(bundle).isDone()); - assertFalse(channel1.getOwnerAsync(bundle).isDone()); - */ FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel2, "producer", producer, true); + } @Test(priority = 6) @@ -514,11 +534,11 @@ public void splitAndRetryTest() throws Exception { Split split = new Split(bundle, ownerAddr1.get(), new HashMap<>()); channel1.publishSplitEventAsync(split); - waitUntilNewOwner(channel1, bundle, null); - waitUntilNewOwner(channel2, bundle, null); + waitUntilState(channel1, bundle, Disabled); + waitUntilState(channel2, bundle, Disabled); - validateHandlerCounters(channel1, 1, 0, 9, 0, 0, 0, 1, 0, 7, 0); - validateHandlerCounters(channel2, 1, 0, 9, 0, 0, 0, 1, 0, 7, 0); + validateHandlerCounters(channel1, 1, 0, 9, 0, 0, 0, 1, 0, 0, 0, 6, 0, 1, 0); + validateHandlerCounters(channel2, 1, 0, 9, 0, 0, 0, 1, 0, 0, 0, 6, 0, 1, 0); validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); // Verify the retry count @@ -538,10 +558,39 @@ public void splitAndRetryTest() throws Exception { assertEquals(Optional.of(lookupServiceAddress1), channel2.getOwnerAsync(childBundle1).get()); assertEquals(Optional.of(lookupServiceAddress1), channel2.getOwnerAsync(childBundle2).get()); + + // try the monitor and check the monitor moves `Disabled` -> `Init` + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); + cleanTableView(channel1, childBundle1); cleanTableView(channel2, childBundle1); cleanTableView(channel1, childBundle2); cleanTableView(channel2, childBundle2); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); } @Test(priority = 7) @@ -774,9 +823,7 @@ public void handleBrokerDeletionEventTest() @Test(priority = 10) public void conflictAndCompactionTest() throws ExecutionException, InterruptedException, TimeoutException, IllegalAccessException, PulsarClientException, PulsarServerException { - - var producer = (Producer) FieldUtils.readDeclaredField(channel1, "producer", true); - producer.newMessage().key(bundle).send(); + String bundle = String.format("%s/%s", "public/default", "0x0000000a_0xffffffff"); var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); assertTrue(owner1.get().isEmpty()); @@ -815,7 +862,7 @@ public void conflictAndCompactionTest() throws ExecutionException, InterruptedEx .untilAsserted(() -> verify(compactor, times(1)) .compact(eq(ServiceUnitStateChannelImpl.TOPIC), any())); - var channel3 = new ServiceUnitStateChannelImpl(pulsar1); + var channel3 = createChannel(pulsar); channel3.start(); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) @@ -830,9 +877,6 @@ public void conflictAndCompactionTest() throws ExecutionException, InterruptedEx @Test(priority = 11) public void ownerLookupCountTests() throws IllegalAccessException { - overrideTableView(channel1, bundle, null); - channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigned, "b1")); channel1.getOwnerAsync(bundle); channel1.getOwnerAsync(bundle); @@ -849,12 +893,73 @@ public void ownerLookupCountTests() throws IllegalAccessException { overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, "b1")); channel1.getOwnerAsync(bundle); - validateOwnerLookUpCounters(channel1, 2, 3, 2, 1, 1); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1")); + channel1.getOwnerAsync(bundle); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Disabled, "b1")); + 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); } + @Test(priority = 12) + public void unloadTest() + throws ExecutionException, InterruptedException, IllegalAccessException { + channel1.publishAssignEventAsync(bundle, lookupServiceAddress1); + waitUntilNewOwner(channel1, bundle, lookupServiceAddress1); + waitUntilNewOwner(channel2, bundle, lookupServiceAddress1); + var ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + var ownerAddr2 = channel2.getOwnerAsync(bundle).get(); - // TODO: add the channel recovery test when broker registry is added. + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(lookupServiceAddress1)); + + Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.empty()); + channel1.publishUnloadEventAsync(unload); + // channel1 is broken. the ownership transfer won't be complete. + waitUntilState(channel1, bundle, Free); + waitUntilState(channel2, bundle, Free); + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertEquals(Optional.empty(), owner1.get()); + assertEquals(Optional.empty(), owner2.get()); + + // test monitor if Free -> Init + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 30 * 1000, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 300 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); + } private static ConcurrentOpenHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { @@ -926,7 +1031,7 @@ private static void waitUntilNewOwner(ServiceUnitStateChannel channel, String se }); } - private static void waitUntilNewState(ServiceUnitStateChannel channel, String key) + private static void waitUntilState(ServiceUnitStateChannel channel, String key) throws IllegalAccessException { TableViewImpl tv = (TableViewImpl) FieldUtils.readField(channel, "tableview", true); @@ -943,6 +1048,20 @@ private static void waitUntilNewState(ServiceUnitStateChannel channel, String ke }); } + private static void waitUntilState(ServiceUnitStateChannel channel, String key, ServiceUnitState expected) + throws IllegalAccessException { + TableViewImpl tv = (TableViewImpl) + FieldUtils.readField(channel, "tableview", true); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(10, TimeUnit.SECONDS) + .until(() -> { // wait until true + ServiceUnitStateData data = tv.get(key); + ServiceUnitState actual = data == null ? Init : data.state(); + return actual == expected; + }); + } + private static void cleanTableView(ServiceUnitStateChannel channel, String serviceUnit) throws IllegalAccessException { var tv = (TableViewImpl) @@ -994,6 +1113,16 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) } } + private void cleanOwnershipMonitorCounters(ServiceUnitStateChannel channel) throws IllegalAccessException { + FieldUtils.writeDeclaredField(channel, "totalCleanupCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalBrokerCleanupTombstoneCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalServiceUnitCleanupTombstoneCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalCleanupErrorCnt", new AtomicLong(0), true); + FieldUtils.writeDeclaredField(channel, "totalCleanupScheduledCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalCleanupIgnoredCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalCleanupCancelledCnt", 0, true); + } + private static long getCleanupMetric(ServiceUnitStateChannel channel, String metric) throws IllegalAccessException { Object var = FieldUtils.readDeclaredField(channel, metric, true); @@ -1009,7 +1138,9 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, long ownedT, long ownedF, long releasedT, long releasedF, long splittingT, long splittingF, - long freeT, long freeF) + long freeT, long freeF, + long initT, long initF, + long disabledT, long disabledF) throws IllegalAccessException { var handlerCounters = (Map) @@ -1029,6 +1160,10 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, assertEquals(splittingF, handlerCounters.get(Splitting).getFailure().get()); assertEquals(freeT, handlerCounters.get(Free).getTotal().get()); assertEquals(freeF, handlerCounters.get(Free).getFailure().get()); + assertEquals(initT, handlerCounters.get(Init).getTotal().get()); + assertEquals(initF, handlerCounters.get(Init).getFailure().get()); + assertEquals(disabledT, handlerCounters.get(Disabled).getTotal().get()); + assertEquals(disabledF, handlerCounters.get(Disabled).getFailure().get()); }); } @@ -1059,7 +1194,10 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, long owned, long released, long splitting, - long free) + long free, + long disabled, + long init + ) throws IllegalAccessException { var ownerLookUpCounters = (Map) @@ -1074,6 +1212,18 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, assertEquals(released, ownerLookUpCounters.get(Released).get()); assertEquals(splitting, ownerLookUpCounters.get(Splitting).get()); assertEquals(free, ownerLookUpCounters.get(Free).get()); + assertEquals(disabled, ownerLookUpCounters.get(Disabled).get()); + assertEquals(init, ownerLookUpCounters.get(Init).get()); }); } + + ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) + throws IllegalAccessException { + var tmpChannel = new ServiceUnitStateChannelImpl(pulsar); + FieldUtils.writeDeclaredField(tmpChannel, "ownershipMonitorDelayTimeInSecs", 5, true); + var channel = spy(tmpChannel); + + doReturn(registry).when(channel).getBrokerRegistry(); + return channel; + } } 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/ServiceUnitStateCompactionStrategyTest.java index 49b55f7660a81..a79f18d9bd5b3 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/ServiceUnitStateCompactionStrategyTest.java @@ -19,7 +19,9 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; @@ -36,7 +38,7 @@ ServiceUnitStateData data(ServiceUnitState state) { } ServiceUnitStateData data(ServiceUnitState state, String dst) { - return new ServiceUnitStateData(state, dst, "broker"); + return new ServiceUnitStateData(state, dst, null); } ServiceUnitStateData data(ServiceUnitState state, String src, String dst) { return new ServiceUnitStateData(state, dst, src); @@ -45,46 +47,76 @@ ServiceUnitStateData data(ServiceUnitState state, String src, String dst) { @Test public void test() throws InterruptedException { String dst = "dst"; + String src = "src"; + assertFalse(strategy.shouldKeepLeft(data(Init), data(Init))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Free))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigned))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Owned))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Released))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Splitting))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Disabled))); + + assertFalse(strategy.shouldKeepLeft(data(Free), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Free))); assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, ""))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned, ""))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Released))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Disabled))); - assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Free))); + assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst2"), data(Owned, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, "src1", dst), data(Owned, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst2"), data(Released, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst1"), data(Owned, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, src, dst))); + assertFalse(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned, src, dst), data(Released, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned, src, "dst1"), data(Released, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Assigned, "src1", dst), data(Released, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Assigned, dst), data(Released, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Splitting))); + assertFalse(strategy.shouldKeepLeft(data(Assigned, src, dst), data(Released, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Splitting, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Disabled, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Assigned, ""))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Assigned, "src", dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned), data(Assigned, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Free, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Free, "dst2"))); + assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Free, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Free, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigned, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst, "dst1"))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Released))); - assertTrue(strategy.shouldKeepLeft(data(Owned,"dst2"), data(Splitting, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Released, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Splitting, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Splitting, "dst2"))); + assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Splitting, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Splitting, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Disabled, dst))); - assertFalse(strategy.shouldKeepLeft(data(Released), data(Free))); + assertFalse(strategy.shouldKeepLeft(data(Released), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Released), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Released, "dst2"), data(Owned, dst))); + assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Owned, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Released, src, "dst1"), data(Owned, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Released, "src1", dst), data(Owned, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Released), data(Owned))); + assertFalse(strategy.shouldKeepLeft(data(Released, src, dst), data(Owned, src, dst))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Released))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Released), data(Disabled, dst))); - assertFalse(strategy.shouldKeepLeft(data(Splitting), data(Free))); + assertFalse(strategy.shouldKeepLeft(data(Splitting), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Owned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Released))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, src, "dst1"), data(Disabled, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, "dst1"), data(Disabled, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, "src1", dst), data(Disabled, "src2", dst))); + assertFalse(strategy.shouldKeepLeft(data(Splitting, dst), data(Disabled, dst))); + assertFalse(strategy.shouldKeepLeft(data(Splitting, src, dst), data(Disabled, src, dst))); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java index 7b9afee9ce2d1..b39f7c2b8d8c6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java @@ -53,11 +53,16 @@ public void testNullState() { new ServiceUnitStateData(null, "A"); } - @Test(expectedExceptions = NullPointerException.class) + @Test(expectedExceptions = IllegalArgumentException.class) public void testNullBroker() { new ServiceUnitStateData(Owned, null); } + @Test(expectedExceptions = IllegalArgumentException.class) + public void testEmptyBroker() { + new ServiceUnitStateData(Owned, ""); + } + @Test public void jsonWriteAndReadTest() throws JsonProcessingException { ObjectMapper mapper = ObjectMapperFactory.create(); 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 69e6a2d204c0e..50e5cad687441 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 @@ -19,7 +19,9 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; @@ -33,35 +35,61 @@ public class ServiceUnitStateTest { @Test public void testTransitions() { + assertTrue(ServiceUnitState.isValidTransition(Init, Init)); + assertTrue(ServiceUnitState.isValidTransition(Init, Free)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); + assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); + assertTrue(ServiceUnitState.isValidTransition(Init, Released)); + assertTrue(ServiceUnitState.isValidTransition(Init, Splitting)); + assertTrue(ServiceUnitState.isValidTransition(Init, Disabled)); + + assertTrue(ServiceUnitState.isValidTransition(Free, Init)); assertFalse(ServiceUnitState.isValidTransition(Free, Free)); assertTrue(ServiceUnitState.isValidTransition(Free, Assigned)); - assertTrue(ServiceUnitState.isValidTransition(Free, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Free, Released)); - assertTrue(ServiceUnitState.isValidTransition(Free, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned)); + assertFalse(ServiceUnitState.isValidTransition(Free, Released)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Free, Disabled)); - assertTrue(ServiceUnitState.isValidTransition(Assigned, Free)); + assertTrue(ServiceUnitState.isValidTransition(Assigned, Init)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Free)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Owned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Released)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Init)); assertTrue(ServiceUnitState.isValidTransition(Owned, Free)); assertTrue(ServiceUnitState.isValidTransition(Owned, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Released)); assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Disabled)); - assertTrue(ServiceUnitState.isValidTransition(Released, Free)); + assertTrue(ServiceUnitState.isValidTransition(Released, Init)); + assertFalse(ServiceUnitState.isValidTransition(Released, Free)); assertFalse(ServiceUnitState.isValidTransition(Released, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Released, Owned)); assertFalse(ServiceUnitState.isValidTransition(Released, Released)); assertFalse(ServiceUnitState.isValidTransition(Released, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Released, Disabled)); - assertTrue(ServiceUnitState.isValidTransition(Splitting, Free)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Init)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Released)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Disabled)); + + assertTrue(ServiceUnitState.isValidTransition(Disabled, Init)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Free)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Owned)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Released)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Disabled)); } } \ No newline at end of file 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 41eaa640d28db..28c3006ca87a6 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 @@ -18,9 +18,11 @@ */ package org.apache.pulsar.compaction; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isValidTransition; import static org.testng.Assert.assertEquals; @@ -42,7 +44,9 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.apache.bookkeeper.client.BookKeeper; import org.apache.commons.lang.reflect.FieldUtils; @@ -81,55 +85,35 @@ public class ServiceUnitStateCompactionTest extends MockedPulsarServiceBaseTest private Schema schema; private ServiceUnitStateCompactionStrategy strategy; - private ServiceUnitState testState0 = Free; - private ServiceUnitState testState1 = Free; - private ServiceUnitState testState2 = Free; - private ServiceUnitState testState3 = Free; - private ServiceUnitState testState4 = Free; + private ServiceUnitState testState = Init; private static Random RANDOM = new Random(); private ServiceUnitStateData testValue(ServiceUnitState state, String broker) { - if (state == Free) { + if (state == Init) { return null; } return new ServiceUnitStateData(state, broker); } - private ServiceUnitStateData testValue0(String broker) { - ServiceUnitState to = nextValidState(testState0); - testState0 = to; + private ServiceUnitStateData testValue(String broker) { + ServiceUnitState to = nextValidStateNonSplit(testState); + testState = to; return testValue(to, broker); } - private ServiceUnitStateData testValue1(String broker) { - ServiceUnitState to = nextValidState(testState1); - testState1 = to; - return testValue(to, broker); - } - - private ServiceUnitStateData testValue2(String broker) { - ServiceUnitState to = nextValidState(testState2); - testState2 = to; - return testValue(to, broker); - } - - private ServiceUnitStateData testValue3(String broker) { - ServiceUnitState to = nextValidState(testState3); - testState3 = to; - return testValue(to, broker); - } - - private ServiceUnitStateData testValue4(String broker) { - ServiceUnitState to = nextValidState(testState4); - testState4 = to; - return testValue(to, broker); + private ServiceUnitState nextValidState(ServiceUnitState from) { + List candidates = Arrays.stream(ServiceUnitState.values()) + .filter(to -> isValidTransition(from, to)) + .collect(Collectors.toList()); + var state= candidates.get(RANDOM.nextInt(candidates.size())); + return state; } - private ServiceUnitState nextValidState(ServiceUnitState from) { + private ServiceUnitState nextValidStateNonSplit(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> to != Free && to != Splitting && isValidTransition(from, to)) + .filter(to -> to != Init && to != Splitting && to != Disabled && isValidTransition(from, to)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; @@ -137,26 +121,14 @@ private ServiceUnitState nextValidState(ServiceUnitState from) { private ServiceUnitState nextInvalidState(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> !isValidTransition(from, to)) + .filter(to -> isValidTransition(from, to)) .collect(Collectors.toList()); if (candidates.size() == 0) { - return null; + return Init; } return candidates.get(RANDOM.nextInt(candidates.size())); } - private List nextStatesToNull(ServiceUnitState from) { - if (from == null) { - return List.of(); - } - return switch (from) { - case Assigned -> List.of(Owned); - case Owned -> List.of(); - case Splitting -> List.of(); - default -> List.of(); - }; - } - @BeforeMethod @Override public void setup() throws Exception { @@ -174,6 +146,7 @@ public void setup() throws Exception { strategy = new ServiceUnitStateCompactionStrategy(); strategy.checkBrokers(false); + testState = Init; } @@ -222,7 +195,7 @@ TestData generateTestData() throws PulsarAdminException, PulsarClientException { int keyIndex = r.nextInt(maxKeys); String key = "key" + keyIndex; ServiceUnitStateData prev = expected.get(key); - ServiceUnitState prevState = prev == null ? Free : prev.state(); + ServiceUnitState prevState = prev == null ? Init : prev.state(); ServiceUnitState state = r.nextBoolean() ? nextInvalidState(prevState) : nextValidState(prevState); ServiceUnitStateData value = new ServiceUnitStateData(state, key + ":" + j); @@ -387,24 +360,26 @@ public void testReadCompactedBeforeCompaction() throws Exception { .create(); pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); - - producer.newMessage().key("key0").value(testValue0( "content0")).send(); - producer.newMessage().key("key0").value(testValue0("content1")).send(); - producer.newMessage().key("key0").value(testValue0( "content2")).send(); + String key = "key0"; + var testValues = Arrays.asList( + testValue("content0"), testValue("content1"), testValue("content2")); + for (var val : testValues) { + producer.newMessage().key(key).value(val).send(); + } try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content0"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(0)); m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content1"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(1)); m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(2)); } StrategicTwoPhaseCompactor compactor @@ -414,8 +389,8 @@ public void testReadCompactedBeforeCompaction() throws Exception { try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(2)); } } @@ -430,30 +405,37 @@ public void testReadEntriesAfterCompaction() throws Exception { pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); - producer.newMessage().key("key0").value(testValue0( "content0")).send(); - producer.newMessage().key("key0").value(testValue0("content1")).send(); - producer.newMessage().key("key0").value(testValue0( "content2")).send(); + String key = "key0"; + var testValues = Arrays.asList( + testValue( "content0"), + testValue("content1"), + testValue( "content2"), + testValue("content3")); + producer.newMessage().key(key).value(testValues.get(0)).send(); + producer.newMessage().key(key).value(testValues.get(1)).send(); + producer.newMessage().key(key).value(testValues.get(2)).send(); StrategicTwoPhaseCompactor compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); compactor.compact(topic, strategy).get(); - producer.newMessage().key("key0").value(testValue0("content3")).send(); + producer.newMessage().key(key).value(testValues.get(3)).send(); try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(2)); m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content3"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(3)); } } @Test public void testSeekEarliestAfterCompaction() throws Exception { + String topic = "persistent://my-property/use/my-ns/my-topic1"; Producer producer = pulsarClient.newProducer(schema) @@ -461,9 +443,14 @@ public void testSeekEarliestAfterCompaction() throws Exception { .enableBatching(true) .create(); - producer.newMessage().key("key0").value(testValue0( "content0")).send(); - producer.newMessage().key("key0").value(testValue0("content1")).send(); - producer.newMessage().key("key0").value(testValue0( "content2")).send(); + String key = "key0"; + var testValues = Arrays.asList( + testValue("content0"), + testValue("content1"), + testValue("content2")); + for (var val : testValues) { + producer.newMessage().key(key).value(val).send(); + } StrategicTwoPhaseCompactor compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); @@ -473,8 +460,8 @@ public void testSeekEarliestAfterCompaction() throws Exception { .readCompacted(true).subscribe()) { consumer.seek(MessageId.earliest); Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(2)); } try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") @@ -482,34 +469,154 @@ public void testSeekEarliestAfterCompaction() throws Exception { consumer.seek(MessageId.earliest); Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content0"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(0)); m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content1"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(1)); m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(2)); } } @Test - public void testBrokerRestartAfterCompaction() throws Exception { + public void testSlowTableviewAfterCompaction() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; + String strategyClassName = "topicCompactionStrategyClassName"; + strategy.checkBrokers(true); + + pulsarClient.newConsumer(schema) + .topic(topic) + .subscriptionName("sub1") + .readCompacted(true) + .subscribe().close(); + + var fastTV = pulsar.getClient().newTableViewBuilder(schema) + .topic(topic) + .subscriptionName("fastTV") + .loadConf(Map.of( + strategyClassName, + ServiceUnitStateCompactionStrategy.class.getName())) + .create(); + + var defaultConf = getDefaultConf(); + var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); + var pulsar2 = additionalPulsarTestContext.getPulsarService(); + + var slowTV = pulsar2.getClient().newTableViewBuilder(schema) + .topic(topic) + .subscriptionName("slowTV") + .loadConf(Map.of( + strategyClassName, + ServiceUnitStateCompactionStrategy.class.getName())) + .create(); + + var semaphore = new Semaphore(0); + AtomicBoolean handledReleased = new AtomicBoolean(false); + + slowTV.listen((k, v) -> { + if (v.state() == Assigned) { + try { + // Stuck at handling Assigned + handledReleased.set(false); + semaphore.acquire(); + //Thread.sleep(5000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } else if (v.state() == Released) { + handledReleased.set(true); + } + }); + + // Configure retention to ensue data is retained for reader + admin.namespaces().setRetention("my-property/use/my-ns", + new RetentionPolicies(-1, -1)); Producer producer = pulsarClient.newProducer(schema) .topic(topic) .enableBatching(true) + .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); - pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); + StrategicTwoPhaseCompactor compactor + = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + + String bundle = "bundle1"; + String src = "broker0"; + String dst = "broker1"; + producer.newMessage().key(bundle).value(new ServiceUnitStateData(Owned, src)).send(); + for (int i = 0; i < 3; i++) { + var assignedStateData = new ServiceUnitStateData(Assigned, dst, src); + producer.newMessage().key(bundle).value(assignedStateData).send(); + producer.newMessage().key(bundle).value(assignedStateData).send(); + var releasedStateData = new ServiceUnitStateData(Released, dst, src); + producer.newMessage().key(bundle).value(releasedStateData).send(); + producer.newMessage().key(bundle).value(releasedStateData).send(); + var ownedStateData = new ServiceUnitStateData(Owned, dst, src); + producer.newMessage().key(bundle).value(ownedStateData).send(); + producer.newMessage().key(bundle).value(ownedStateData).send(); + compactor.compact(topic, strategy).get(); + + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals(fastTV.get(bundle), ownedStateData)); + + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals(slowTV.get(bundle), assignedStateData)); + assertTrue(!handledReleased.get()); + semaphore.release(); + + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals(slowTV.get(bundle), ownedStateData)); + + var newTv = pulsar.getClient().newTableView(schema) + .topic(topic) + .loadConf(Map.of( + strategyClassName, + ServiceUnitStateCompactionStrategy.class.getName())) + .create(); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals(newTv.get(bundle), ownedStateData)); + + src = dst; + dst = "broker" + (i + 2); + newTv.close(); + } + + producer.close(); + slowTV.close(); + fastTV.close(); + pulsar2.close(); + + } - producer.newMessage().key("key0").value(testValue0( "content0")).send(); - producer.newMessage().key("key0").value(testValue0("content1")).send(); - producer.newMessage().key("key0").value(testValue0( "content2")).send(); + @Test + public void testBrokerRestartAfterCompaction() throws Exception { + String topic = "persistent://my-property/use/my-ns/my-topic1"; + Producer producer = pulsarClient.newProducer(schema) + .topic(topic) + .enableBatching(true) + .create(); + String key = "key0"; + pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); + + var testValues = Arrays.asList( + testValue("content0"), testValue("content1"), testValue("content2")); + for (var val : testValues) { + producer.newMessage().key(key).value(val).send(); + } StrategicTwoPhaseCompactor compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); compactor.compact(topic, strategy).get(); @@ -517,8 +624,8 @@ public void testBrokerRestartAfterCompaction() throws Exception { try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(testValues.size() - 1)); } stopBroker(); @@ -534,8 +641,8 @@ public void testBrokerRestartAfterCompaction() throws Exception { try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); - Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content2"); + Assert.assertEquals(m.getKey(), key); + Assert.assertEquals(m.getValue(), testValues.get(testValues.size() - 1)); } } @@ -554,13 +661,14 @@ public void testCompactEmptyTopic() throws Exception { = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); compactor.compact(topic, strategy).get(); - producer.newMessage().key("key0").value(testValue0( "content0")).send(); + var testValue = testValue( "content0"); + producer.newMessage().key("key0").value(testValue).send(); try (Consumer consumer = pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1") .readCompacted(true).subscribe()) { Message m = consumer.receive(); Assert.assertEquals(m.getKey(), "key0"); - Assert.assertEquals(m.getValue().broker(), "content0"); + Assert.assertEquals(m.getValue(), testValue); } } @@ -583,10 +691,10 @@ public void testWholeBatchCompactedOut() throws Exception { .batchingMaxPublishDelay(1, TimeUnit.HOURS) .messageRoutingMode(MessageRoutingMode.SinglePartition) .create()) { - producerBatch.newMessage().key("key1").value(testValue1("my-message-1")).sendAsync(); - producerBatch.newMessage().key("key1").value(testValue1( "my-message-2")).sendAsync(); - producerBatch.newMessage().key("key1").value(testValue1("my-message-3")).sendAsync(); - producerNormal.newMessage().key("key1").value(testValue1( "my-message-4")).send(); + producerBatch.newMessage().key("key1").value(testValue("my-message-1")).sendAsync(); + producerBatch.newMessage().key("key1").value(testValue( "my-message-2")).sendAsync(); + producerBatch.newMessage().key("key1").value(testValue("my-message-3")).sendAsync(); + producerNormal.newMessage().key("key1").value(testValue( "my-message-4")).send(); } // compact the topic @@ -707,7 +815,7 @@ public void testCompactMultipleTimesWithoutEmptyMessage() List> futures = new ArrayList<>(messages); for (int i = 0; i < messages; i++) { - futures.add(producer.newMessage().key(key).value(testValue0((i + ""))).sendAsync()); + futures.add(producer.newMessage().key(key).value(testValue((i + ""))).sendAsync()); } FutureUtil.waitForAll(futures).get(); @@ -720,7 +828,7 @@ public void testCompactMultipleTimesWithoutEmptyMessage() // 3. Send more ten messages futures.clear(); for (int i = 0; i < messages; i++) { - futures.add(producer.newMessage().key(key).value(testValue0((i + 10 + ""))).sendAsync()); + futures.add(producer.newMessage().key(key).value(testValue((i + 10 + ""))).sendAsync()); } FutureUtil.waitForAll(futures).get(); @@ -754,7 +862,7 @@ public void testReadUnCompacted() List> futures = new ArrayList<>(messages); for (int i = 0; i < messages; i++) { - futures.add(producer.newMessage().key(key).value(testValue0((i + ""))).sendAsync()); + futures.add(producer.newMessage().key(key).value(testValue((i + ""))).sendAsync()); } FutureUtil.waitForAll(futures).get(); @@ -767,7 +875,7 @@ public void testReadUnCompacted() // 3. Send more ten messages futures.clear(); for (int i = 0; i < messages; i++) { - futures.add(producer.newMessage().key(key).value(testValue0((i + 10 + ""))).sendAsync()); + futures.add(producer.newMessage().key(key).value(testValue((i + 10 + ""))).sendAsync()); } FutureUtil.waitForAll(futures).get(); try (Consumer consumer = pulsarClient.newConsumer(schema) @@ -788,9 +896,6 @@ public void testReadUnCompacted() } // 4.Send empty message to delete the key-value in the compacted topic. - for (ServiceUnitState state : nextStatesToNull(testState0)) { - producer.newMessage().key(key).value(new ServiceUnitStateData(state, "xx")).send(); - } producer.newMessage().key(key).value(null).send(); // 5.compact the topic. @@ -807,7 +912,7 @@ public void testReadUnCompacted() } for (int i = 0; i < messages; i++) { - futures.add(producer.newMessage().key(key).value(testValue0((i + 20 + ""))).sendAsync()); + futures.add(producer.newMessage().key(key).value(testValue((i + 20 + ""))).sendAsync()); } FutureUtil.waitForAll(futures).get(); From d52884408fb45bd5d9a323c769481ec4a213a2d1 Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Fri, 17 Feb 2023 11:37:58 -0800 Subject: [PATCH 2/7] Made unload command 2pc --- .../extensions/channel/ServiceUnitState.java | 14 ++--- .../channel/ServiceUnitStateChannelImpl.java | 26 +++++--- .../ServiceUnitStateCompactionStrategy.java | 17 +++--- .../ExtensibleLoadManagerImplTest.java | 16 ++--- .../channel/ServiceUnitStateChannelTest.java | 57 +++++------------- ...erviceUnitStateCompactionStrategyTest.java | 59 +++++++++++-------- .../channel/ServiceUnitStateTest.java | 50 ++++++++-------- .../ServiceUnitStateCompactionTest.java | 4 +- 8 files changed, 115 insertions(+), 128 deletions(-) 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 5043aa84936ec..7aff231277b28 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 @@ -30,7 +30,7 @@ public enum ServiceUnitState { Init, // initializing the state. no previous state(terminal state) - Free, // not owned by any broker (semi-terminal state) + Disabled, // disabled by the owner broker Owned, // owned by a broker (terminal state) @@ -40,18 +40,18 @@ public enum ServiceUnitState { Splitting, // the service unit(e.g. bundle) is in the process of splitting. - Disabled; // disabled in the system (semi-terminal state) + Deleted; // deleted in the system (semi-terminal state) private static Map> validTransitions = 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, Assigned, Released, Splitting, Disabled, Init), - Free, Set.of(Assigned, Init), - Owned, Set.of(Assigned, Splitting, Free, Init), + Init, Set.of(Disabled, Owned, Assigned, Released, Splitting, Deleted, Init), + Disabled, Set.of(Init), + Owned, Set.of(Assigned, Splitting, Disabled, Init), Assigned, Set.of(Owned, Released, Init), Released, Set.of(Owned, Init), - Splitting, Set.of(Disabled, Init), - Disabled, Set.of(Init) + Splitting, Set.of(Deleted, Init), + Deleted, Set.of(Init) ); public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to) { 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 051e19b61ea4f..4c78593062570 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 @@ -22,8 +22,8 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -425,14 +425,14 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { case Owned, Splitting -> { return CompletableFuture.completedFuture(Optional.of(data.broker())); } - case Assigned, Released -> { + case Assigned, Released, Disabled -> { return deferGetOwnerRequest(serviceUnit).thenApply(Optional::of); } - case Free, Init -> { + case Init -> { return CompletableFuture.completedFuture(Optional.empty()); } - case Disabled -> { - return CompletableFuture.failedFuture(new IllegalArgumentException(serviceUnit + " is disabled")); + case Deleted -> { + return CompletableFuture.failedFuture(new IllegalArgumentException(serviceUnit + " is deleted.")); } default -> { String errorMsg = String.format("Failed to process service unit state data: %s when get owner.", data); @@ -469,7 +469,7 @@ public CompletableFuture publishUnloadEventAsync(Unload unload) { Assigned, unload.destBroker().get(), unload.sourceBroker())); } else { future = pubAsync(serviceUnit, new ServiceUnitStateData( - Free, unload.sourceBroker())); + Disabled, unload.sourceBroker())); } return future.whenComplete((__, ex) -> { @@ -505,7 +505,8 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { case Assigned -> handleAssignEvent(serviceUnit, data); case Released -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); - case Free, Disabled, Init -> handleInitEvent(serviceUnit); + case Disabled -> handleDisableEvent(serviceUnit, data); + case Deleted, Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } } catch (Throwable e){ @@ -612,6 +613,13 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { } } + private void handleDisableEvent(String serviceUnit, ServiceUnitStateData data) { + if (isTargetBroker(data.broker())) { + tombstoneAsync(serviceUnit) + .whenComplete((__, e) -> log(e, serviceUnit, data, null)); + } + } + private void handleInitEvent(String serviceUnit) { closeServiceUnit(serviceUnit) .thenAccept(__ -> { @@ -781,7 +789,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, updateFuture.thenAccept(r -> { // Disable the old bundle - pubAsync(serviceUnit, new ServiceUnitStateData(Disabled, data.broker())).thenRun(() -> { + pubAsync(serviceUnit, new ServiceUnitStateData(Deleted, data.broker())).thenRun(() -> { // Update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(bundle); // TODO: Update the load data immediately if needed. @@ -979,7 +987,7 @@ protected void monitorOwnerships(List brokers) { } else if (state != Owned && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { boolean tombstone = false; - if ((state == Free || state == Disabled)) { + if (state == Deleted) { if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { log.info("Found semi-terminal states(free or disabled) to clean" 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/ServiceUnitStateCompactionStrategy.java index 19f928b23bae1..34c400f4ba9d5 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/ServiceUnitStateCompactionStrategy.java @@ -56,25 +56,20 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to if (checkBrokers) { switch (prevState) { - case Free: - switch (state) { - case Assigned: - return isNotBlank(to.sourceBroker()); - } case Owned: switch (state) { case Assigned: return invalidTransfer(from, to); case Splitting: - case Free: - return !from.broker().equals(to.broker()); + case Disabled: + return targetNotEquals(from, to); } case Assigned: switch (state) { case Released: return isBlank(to.sourceBroker()) || notEquals(from, to); case Owned: - return isNotBlank(to.sourceBroker()) || !to.broker().equals(from.broker()); + return isNotBlank(to.sourceBroker()) || targetNotEquals(from, to); } case Released: switch (state) { @@ -83,7 +78,7 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to } case Splitting: switch (state) { - case Disabled : + case Deleted: return notEquals(from, to); } } @@ -91,6 +86,10 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to return false; } + private boolean targetNotEquals(ServiceUnitStateData from, ServiceUnitStateData to) { + return !from.broker().equals(to.broker()); + } + private boolean notEquals(ServiceUnitStateData from, ServiceUnitStateData to) { return !from.broker().equals(to.broker()) || !StringUtils.equals(from.sourceBroker(), to.sourceBroker()); 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 19485b25fb4cc..d9f2a2d49d020 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 @@ -431,6 +431,7 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] @@ -440,7 +441,6 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] @@ -451,16 +451,16 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundlesSplit, reason=Admin, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=5}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] + dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Disabled}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Disabled}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=1}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=2}] dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=3}] @@ -469,8 +469,8 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=6}] dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=1}] dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=2}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] + dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] + dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] @@ -479,8 +479,8 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] - dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] - dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=4}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=6}] 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 148ca982b11c7..4ee7a7d1ee041 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 @@ -19,8 +19,8 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -534,8 +534,8 @@ public void splitAndRetryTest() throws Exception { Split split = new Split(bundle, ownerAddr1.get(), new HashMap<>()); channel1.publishSplitEventAsync(split); - waitUntilState(channel1, bundle, Disabled); - waitUntilState(channel2, bundle, Disabled); + waitUntilState(channel1, bundle, Deleted); + waitUntilState(channel2, bundle, Deleted); validateHandlerCounters(channel1, 1, 0, 9, 0, 0, 0, 1, 0, 0, 0, 6, 0, 1, 0); validateHandlerCounters(channel2, 1, 0, 9, 0, 0, 0, 1, 0, 0, 0, 6, 0, 1, 0); @@ -893,10 +893,10 @@ public void ownerLookupCountTests() throws IllegalAccessException { overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, "b1")); channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1")); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Disabled, "b1")); channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Disabled, "b1")); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1")); channel1.getOwnerAsync(bundle); channel1.getOwnerAsync(bundle); @@ -920,45 +920,16 @@ public void unloadTest() assertEquals(ownerAddr1, ownerAddr2); assertEquals(ownerAddr1, Optional.of(lookupServiceAddress1)); - Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.empty()); channel1.publishUnloadEventAsync(unload); - // channel1 is broken. the ownership transfer won't be complete. - waitUntilState(channel1, bundle, Free); - waitUntilState(channel2, bundle, Free); + + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); assertEquals(Optional.empty(), owner1.get()); assertEquals(Optional.empty(), owner2.get()); - - // test monitor if Free -> Init - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); - FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 1, true); - - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); - FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 1, true); - - ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( - List.of(lookupServiceAddress1, lookupServiceAddress2)); - ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( - List.of(lookupServiceAddress1, lookupServiceAddress2)); - waitUntilState(channel1, bundle, Init); - waitUntilState(channel2, bundle, Init); - - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel1, - "semiTerminalStateWaitingTimeInMillis", 30 * 1000, true); - - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 300 * 1000, true); - FieldUtils.writeDeclaredField(channel2, - "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); } private static ConcurrentOpenHashMap>> getOwnerRequests( @@ -1158,12 +1129,12 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, assertEquals(releasedF, handlerCounters.get(Released).getFailure().get()); assertEquals(splittingT, handlerCounters.get(Splitting).getTotal().get()); assertEquals(splittingF, handlerCounters.get(Splitting).getFailure().get()); - assertEquals(freeT, handlerCounters.get(Free).getTotal().get()); - assertEquals(freeF, handlerCounters.get(Free).getFailure().get()); + assertEquals(freeT, handlerCounters.get(Disabled).getTotal().get()); + assertEquals(freeF, handlerCounters.get(Disabled).getFailure().get()); assertEquals(initT, handlerCounters.get(Init).getTotal().get()); assertEquals(initF, handlerCounters.get(Init).getFailure().get()); - assertEquals(disabledT, handlerCounters.get(Disabled).getTotal().get()); - assertEquals(disabledF, handlerCounters.get(Disabled).getFailure().get()); + assertEquals(disabledT, handlerCounters.get(Deleted).getTotal().get()); + assertEquals(disabledF, handlerCounters.get(Deleted).getFailure().get()); }); } @@ -1211,8 +1182,8 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, assertEquals(owned, ownerLookUpCounters.get(Owned).get()); assertEquals(released, ownerLookUpCounters.get(Released).get()); assertEquals(splitting, ownerLookUpCounters.get(Splitting).get()); - assertEquals(free, ownerLookUpCounters.get(Free).get()); - assertEquals(disabled, ownerLookUpCounters.get(Disabled).get()); + assertEquals(free, ownerLookUpCounters.get(Disabled).get()); + assertEquals(disabled, ownerLookUpCounters.get(Deleted).get()); assertEquals(init, ownerLookUpCounters.get(Init).get()); }); } 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/ServiceUnitStateCompactionStrategyTest.java index a79f18d9bd5b3..df9033757a5e6 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/ServiceUnitStateCompactionStrategyTest.java @@ -19,8 +19,8 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -49,24 +49,23 @@ public void test() throws InterruptedException { String dst = "dst"; String src = "src"; assertFalse(strategy.shouldKeepLeft(data(Init), data(Init))); - assertFalse(strategy.shouldKeepLeft(data(Init), data(Free))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Disabled))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigned))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Owned))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Released))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Splitting))); - assertFalse(strategy.shouldKeepLeft(data(Init), data(Disabled))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Deleted))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Free))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, src, dst))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Released))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Splitting))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Disabled))); + assertFalse(strategy.shouldKeepLeft(data(Disabled), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Assigned))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Owned))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Deleted))); assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Disabled))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst1"), data(Owned, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, src, dst))); @@ -76,13 +75,13 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Assigned, "src1", dst), data(Released, "src2", dst))); assertFalse(strategy.shouldKeepLeft(data(Assigned, src, dst), data(Released, src, dst))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Splitting, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Disabled, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Owned), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Free, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Free, "dst2"))); - assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Free, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Free, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Disabled, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Disabled, "dst2"))); + assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Disabled, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Disabled, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigned, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, src, dst))); @@ -94,10 +93,10 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Splitting, "dst2"))); assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Splitting, dst))); assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Splitting, dst))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Disabled, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Released), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Released), data(Disabled))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Owned, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Released, src, "dst1"), data(Owned, src, "dst2"))); @@ -105,18 +104,26 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Released, src, dst), data(Owned, src, dst))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Released))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Splitting))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Disabled, dst))); + assertTrue(strategy.shouldKeepLeft(data(Released), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Splitting), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Disabled))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Owned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Released))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Splitting))); - assertTrue(strategy.shouldKeepLeft(data(Splitting, src, "dst1"), data(Disabled, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Splitting, "dst1"), data(Disabled, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Splitting, "src1", dst), data(Disabled, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Splitting, dst), data(Disabled, dst))); - assertFalse(strategy.shouldKeepLeft(data(Splitting, src, dst), data(Disabled, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, src, "dst1"), data(Deleted, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, "dst1"), data(Deleted, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Splitting, "src1", dst), data(Deleted, "src2", dst))); + assertFalse(strategy.shouldKeepLeft(data(Splitting, dst), data(Deleted, dst))); + assertFalse(strategy.shouldKeepLeft(data(Splitting, src, dst), data(Deleted, src, dst))); + + assertFalse(strategy.shouldKeepLeft(data(Deleted), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Assigned))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Owned))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Deleted))); } } 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 50e5cad687441..288b683086c08 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 @@ -19,8 +19,8 @@ package org.apache.pulsar.broker.loadbalance.extensions.channel; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -36,60 +36,60 @@ public class ServiceUnitStateTest { public void testTransitions() { assertTrue(ServiceUnitState.isValidTransition(Init, Init)); - assertTrue(ServiceUnitState.isValidTransition(Init, Free)); + assertTrue(ServiceUnitState.isValidTransition(Init, Disabled)); assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); assertTrue(ServiceUnitState.isValidTransition(Init, Released)); assertTrue(ServiceUnitState.isValidTransition(Init, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Init, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Init, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Free, Init)); - assertFalse(ServiceUnitState.isValidTransition(Free, Free)); - assertTrue(ServiceUnitState.isValidTransition(Free, Assigned)); - assertFalse(ServiceUnitState.isValidTransition(Free, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Free, Released)); - assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Free, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Disabled, Init)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Owned)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Released)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Disabled, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Init)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Free)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Disabled)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Owned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Released)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Owned, Init)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Free)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Disabled)); assertTrue(ServiceUnitState.isValidTransition(Owned, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Released)); assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Released, Init)); - assertFalse(ServiceUnitState.isValidTransition(Released, Free)); + assertFalse(ServiceUnitState.isValidTransition(Released, Disabled)); assertFalse(ServiceUnitState.isValidTransition(Released, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Released, Owned)); assertFalse(ServiceUnitState.isValidTransition(Released, Released)); assertFalse(ServiceUnitState.isValidTransition(Released, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Released, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Released, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Splitting, Init)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Disabled)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Released)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Splitting, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Disabled, Init)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Free)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Assigned)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Released)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Deleted, Init)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Released)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted)); } } \ No newline at end of file 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 28c3006ca87a6..a0648bfb729a8 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 @@ -18,6 +18,7 @@ */ package org.apache.pulsar.compaction; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; @@ -113,7 +114,8 @@ private ServiceUnitState nextValidState(ServiceUnitState from) { private ServiceUnitState nextValidStateNonSplit(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> to != Init && to != Splitting && to != Disabled && isValidTransition(from, to)) + .filter(to -> to != Init && to != Splitting && to != Deleted && to != Disabled + && isValidTransition(from, to)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; From c485357e1532de90d9b429e394f57b85e7499d8a Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Fri, 17 Feb 2023 14:11:46 -0800 Subject: [PATCH 3/7] Added Free state --- .../extensions/channel/ServiceUnitState.java | 5 +- .../channel/ServiceUnitStateChannelImpl.java | 16 +++--- .../ServiceUnitStateCompactionStrategy.java | 10 ++++ .../ExtensibleLoadManagerImplTest.java | 37 +++++++------ .../channel/ServiceUnitStateChannelTest.java | 53 ++++++++++++++++++- ...erviceUnitStateCompactionStrategyTest.java | 10 ++++ .../ServiceUnitStateCompactionTest.java | 3 +- 7 files changed, 105 insertions(+), 29 deletions(-) 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 7aff231277b28..dff5dad23a3e9 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 @@ -32,6 +32,8 @@ public enum ServiceUnitState { Disabled, // disabled by the owner broker + Free, // not owned by any broker (semi-terminal state) + Owned, // owned by a broker (terminal state) Assigned, // the ownership is assigned(but the assigned broker has not been notified the ownership yet) @@ -46,7 +48,8 @@ public enum ServiceUnitState { // (Init -> all states) transitions are required // when the topic is compacted in the middle of assign, transfer or split. Init, Set.of(Disabled, Owned, Assigned, Released, Splitting, Deleted, Init), - Disabled, Set.of(Init), + Disabled, Set.of(Free, Init), + Free, Set.of(Assigned, Init), Owned, Set.of(Assigned, Splitting, Disabled, Init), Assigned, Set.of(Owned, Released, Init), Released, Set.of(Owned, Init), 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 4c78593062570..8359ea27f8540 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 @@ -24,6 +24,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -428,7 +429,7 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { case Assigned, Released, Disabled -> { return deferGetOwnerRequest(serviceUnit).thenApply(Optional::of); } - case Init -> { + case Init, Free -> { return CompletableFuture.completedFuture(Optional.empty()); } case Deleted -> { @@ -506,7 +507,7 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { case Released -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); case Disabled -> handleDisableEvent(serviceUnit, data); - case Deleted, Init -> handleInitEvent(serviceUnit); + case Deleted, Free, Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } } catch (Throwable e){ @@ -615,8 +616,9 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { private void handleDisableEvent(String serviceUnit, ServiceUnitStateData data) { if (isTargetBroker(data.broker())) { - tombstoneAsync(serviceUnit) - .whenComplete((__, e) -> log(e, serviceUnit, data, null)); + ServiceUnitStateData next = new ServiceUnitStateData(Free, data.broker()); + pubAsync(serviceUnit, next) + .whenComplete((__, e) -> log(e, serviceUnit, data, next)); } } @@ -788,7 +790,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, }); updateFuture.thenAccept(r -> { - // Disable the old bundle + // Delete the old bundle pubAsync(serviceUnit, new ServiceUnitStateData(Deleted, data.broker())).thenRun(() -> { // Update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(bundle); @@ -987,10 +989,10 @@ protected void monitorOwnerships(List brokers) { } else if (state != Owned && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { boolean tombstone = false; - if (state == Deleted) { + if (state == Deleted || state == Free) { if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { - log.info("Found semi-terminal states(free or disabled) to clean" + log.info("Found semi-terminal states to clean" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); tombstone = true; } 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/ServiceUnitStateCompactionStrategy.java index 34c400f4ba9d5..795167bcfa112 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/ServiceUnitStateCompactionStrategy.java @@ -81,6 +81,16 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to case Deleted: return notEquals(from, to); } + case Disabled: + switch (state) { + case Free: + return notEquals(from, to); + } + case Free: + switch (state) { + case Assigned: + return isNotBlank(to.sourceBroker()); + } } } return false; 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 d9f2a2d49d020..c9a9ba7ad72ff 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 @@ -431,7 +431,6 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] @@ -441,6 +440,7 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] @@ -451,16 +451,17 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundlesSplit, reason=Admin, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=5}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] - dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] + dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Disabled}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=8}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=1}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=2}] dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=3}] @@ -471,16 +472,18 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=2}] dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] - dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] - dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=15}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=16}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=4}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=6}] 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 4ee7a7d1ee041..163c1173ea9fc 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 @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -912,7 +913,9 @@ public void ownerLookupCountTests() throws IllegalAccessException { @Test(priority = 12) public void unloadTest() throws ExecutionException, InterruptedException, IllegalAccessException { + channel1.publishAssignEventAsync(bundle, lookupServiceAddress1); + waitUntilNewOwner(channel1, bundle, lookupServiceAddress1); waitUntilNewOwner(channel2, bundle, lookupServiceAddress1); var ownerAddr1 = channel1.getOwnerAsync(bundle).get(); @@ -921,15 +924,61 @@ public void unloadTest() assertEquals(ownerAddr1, ownerAddr2); assertEquals(ownerAddr1, Optional.of(lookupServiceAddress1)); Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.empty()); + channel1.publishUnloadEventAsync(unload); - waitUntilState(channel1, bundle, Init); - waitUntilState(channel2, bundle, Init); + waitUntilState(channel1, bundle, Free); + waitUntilState(channel2, bundle, Free); var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); assertEquals(Optional.empty(), owner1.get()); assertEquals(Optional.empty(), owner2.get()); + + channel2.publishAssignEventAsync(bundle, lookupServiceAddress2); + + waitUntilNewOwner(channel1, bundle, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle, lookupServiceAddress2); + + ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(lookupServiceAddress2)); + Unload unload2 = new Unload(lookupServiceAddress2, bundle, Optional.empty()); + + channel2.publishUnloadEventAsync(unload2); + + waitUntilState(channel1, bundle, Free); + waitUntilState(channel2, bundle, Free); + + // test monitor if Free -> Init + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 1, true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel1, + "semiTerminalStateWaitingTimeInMillis", 30 * 1000, true); + + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 300 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); } private static ConcurrentOpenHashMap>> getOwnerRequests( 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/ServiceUnitStateCompactionStrategyTest.java index df9033757a5e6..cfdfe7c2c71a4 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/ServiceUnitStateCompactionStrategyTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -125,5 +126,14 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Released))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Splitting))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Deleted))); + + assertFalse(strategy.shouldKeepLeft(data(Free), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Disabled))); + assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigned))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Deleted))); } } 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 a0648bfb729a8..568f8fd8e50c6 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 @@ -19,7 +19,6 @@ package org.apache.pulsar.compaction; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; @@ -114,7 +113,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 && to != Disabled + .filter(to -> to != Init && to != Splitting && to != Deleted && isValidTransition(from, to)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); From 6d7316da7e8f1fe9bf85e79a90bb52bc73a39e06 Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Mon, 20 Feb 2023 00:08:48 -0800 Subject: [PATCH 4/7] Removed Disabled --- .../pulsar/broker/ServiceConfiguration.java | 6 +-- .../extensions/channel/ServiceUnitState.java | 9 ++-- .../channel/ServiceUnitStateChannelImpl.java | 38 ++++++++------- .../ServiceUnitStateCompactionStrategy.java | 10 ++-- .../ExtensibleLoadManagerImplTest.java | 47 +++++++++---------- .../channel/ServiceUnitStateChannelTest.java | 19 ++++---- ...erviceUnitStateCompactionStrategyTest.java | 32 +++++-------- .../channel/ServiceUnitStateTest.java | 32 ++++++------- .../ServiceUnitStateCompactionTest.java | 2 +- 9 files changed, 91 insertions(+), 104 deletions(-) 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 a846cc002d39f..4f2c8e72e131d 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 @@ -2529,12 +2529,12 @@ The delayed message index bucket time step(in seconds) in per bucket snapshot se @FieldContext( category = CATEGORY_LOAD_BALANCER, doc = "After this delay, the service-unit state channel tombstones any service units (e.g., bundles) " - + "in semi-terminal states. For example, after splits, parent bundles will be `Disabled`, " - + "and then after this delay, the parent bundles' state will be removed(tombstoned) " + + "in semi-terminal states. For example, after splits, parent bundles will be `deleted`, " + + "and then after this delay, the parent bundles' state will be `tombstoned` " + "in the service-unit state channel. " + "Pulsar does not immediately remove such semi-terminal states " + "to avoid unnecessary system confusion, " - + "as the bundles in the `Tombstoned` state might temporarily look available to reassign. " + + "as the bundles in the `tombstoned` state might temporarily look available to reassign. " + "Rarely, one could lower this delay in order to aggressively clean " + "the service-unit state channel when there are a large number of bundles. " + "minimum value = 30 secs" 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 dff5dad23a3e9..40dd87c85d711 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 @@ -30,8 +30,6 @@ public enum ServiceUnitState { Init, // initializing the state. no previous state(terminal state) - Disabled, // disabled by the owner broker - Free, // not owned by any broker (semi-terminal state) Owned, // owned by a broker (terminal state) @@ -47,12 +45,11 @@ public enum ServiceUnitState { private static Map> validTransitions = Map.of( // (Init -> all states) transitions are required // when the topic is compacted in the middle of assign, transfer or split. - Init, Set.of(Disabled, Owned, Assigned, Released, Splitting, Deleted, Init), - Disabled, Set.of(Free, Init), + Init, Set.of(Free, Owned, Assigned, Released, Splitting, Deleted, Init), Free, Set.of(Assigned, Init), - Owned, Set.of(Assigned, Splitting, Disabled, Init), + Owned, Set.of(Assigned, Splitting, Released, Init), Assigned, Set.of(Owned, Released, Init), - Released, Set.of(Owned, Init), + Released, Set.of(Owned, Free, Init), Splitting, Set.of(Deleted, Init), Deleted, Set.of(Init) ); 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 8359ea27f8540..9ff272c1e1f76 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 @@ -23,7 +23,6 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; @@ -248,13 +247,7 @@ public synchronized void start() throws PulsarServerException { try { this.brokerRegistry = getBrokerRegistry(); this.brokerRegistry.addListener((broker, type) -> { - if (type == NotificationType.Deleted) { - log.info("BrokerRegistry detected the broker:{} registry has been deleted.", broker); - handleBrokerDeletionEvent(broker); - } else if (type == NotificationType.Created) { - log.info("BrokerRegistry detected the broker:{} registry has been created.", broker); - handleBrokerCreationEvent(broker); - } + handleBrokerRegistrationEvent(broker, type); }); leaderElectionService.start(); this.channelState = LeaderElectionServiceStarted; @@ -426,7 +419,7 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { case Owned, Splitting -> { return CompletableFuture.completedFuture(Optional.of(data.broker())); } - case Assigned, Released, Disabled -> { + case Assigned, Released -> { return deferGetOwnerRequest(serviceUnit).thenApply(Optional::of); } case Init, Free -> { @@ -470,7 +463,7 @@ public CompletableFuture publishUnloadEventAsync(Unload unload) { Assigned, unload.destBroker().get(), unload.sourceBroker())); } else { future = pubAsync(serviceUnit, new ServiceUnitStateData( - Disabled, unload.sourceBroker())); + Released, unload.sourceBroker())); } return future.whenComplete((__, ex) -> { @@ -506,7 +499,6 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { case Assigned -> handleAssignEvent(serviceUnit, data); case Released -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); - case Disabled -> handleDisableEvent(serviceUnit, data); case Deleted, Free, Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } @@ -598,12 +590,22 @@ private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) { } private void handleReleaseEvent(String serviceUnit, ServiceUnitStateData data) { - if (isTargetBroker(data.sourceBroker())) { - ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker(), data.sourceBroker()); - // TODO: when close, pass message to clients to connect to the new broker - closeServiceUnit(serviceUnit) - .thenCompose(__ -> pubAsync(serviceUnit, next)) - .whenComplete((__, e) -> log(e, serviceUnit, data, next)); + + if (isTransferCommand(data)) { + if (isTargetBroker(data.sourceBroker())) { + ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker(), data.sourceBroker()); + // TODO: when close, pass message to clients to connect to the new broker + closeServiceUnit(serviceUnit) + .thenCompose(__ -> pubAsync(serviceUnit, next)) + .whenComplete((__, e) -> log(e, serviceUnit, data, next)); + } + } else { + if (isTargetBroker(data.broker())) { + ServiceUnitStateData next = new ServiceUnitStateData(Free, data.broker()); + closeServiceUnit(serviceUnit) + .thenCompose(__ -> pubAsync(serviceUnit, next)) + .whenComplete((__, e) -> log(e, serviceUnit, data, next)); + } } } @@ -838,8 +840,10 @@ public void handleMetadataSessionEvent(SessionEvent e) { public void handleBrokerRegistrationEvent(String broker, NotificationType type) { if (type == NotificationType.Created) { + log.info("BrokerRegistry detected the broker:{} registry has been created.", broker); handleBrokerCreationEvent(broker); } else if (type == NotificationType.Deleted) { + log.info("BrokerRegistry detected the broker:{} registry has been deleted.", broker); handleBrokerDeletionEvent(broker); } } 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/ServiceUnitStateCompactionStrategy.java index 795167bcfa112..0833569cf5f1a 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/ServiceUnitStateCompactionStrategy.java @@ -61,8 +61,8 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to case Assigned: return invalidTransfer(from, to); case Splitting: - case Disabled: - return targetNotEquals(from, to); + case Released: + return isNotBlank(to.sourceBroker()) || targetNotEquals(from, to); } case Assigned: switch (state) { @@ -74,6 +74,7 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to case Released: switch (state) { case Owned: + case Free: return notEquals(from, to); } case Splitting: @@ -81,11 +82,6 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to case Deleted: return notEquals(from, to); } - case Disabled: - switch (state) { - case Free: - return notEquals(from, to); - } case Free: switch (state) { case Assigned: 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 c9a9ba7ad72ff..8d386371b6bf4 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 @@ -431,6 +431,9 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] @@ -438,9 +441,6 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoLoadData, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=7}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBrokers, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=8}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] @@ -452,16 +452,15 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] - dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Disabled}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=8}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=1}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=2}] dimensions=[{broker=localhost, event=Split, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=3}] @@ -470,20 +469,18 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Unload, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=6}] dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=1}] dimensions=[{broker=localhost, event=Init, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=2}] - dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] - dimensions=[{broker=localhost, event=Disabled, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] - dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] - dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] - dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] - dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=15}] - dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=16}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=3}] + dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] + dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] + dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] + dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] + dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] + dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=4}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=6}] 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 163c1173ea9fc..663c6e295c87d 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 @@ -20,7 +20,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; @@ -894,7 +893,7 @@ public void ownerLookupCountTests() throws IllegalAccessException { overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, "b1")); channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Disabled, "b1")); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1")); channel1.getOwnerAsync(bundle); overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1")); @@ -1160,7 +1159,7 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, long splittingT, long splittingF, long freeT, long freeF, long initT, long initF, - long disabledT, long disabledF) + long deletedT, long deletedF) throws IllegalAccessException { var handlerCounters = (Map) @@ -1178,12 +1177,12 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, assertEquals(releasedF, handlerCounters.get(Released).getFailure().get()); assertEquals(splittingT, handlerCounters.get(Splitting).getTotal().get()); assertEquals(splittingF, handlerCounters.get(Splitting).getFailure().get()); - assertEquals(freeT, handlerCounters.get(Disabled).getTotal().get()); - assertEquals(freeF, handlerCounters.get(Disabled).getFailure().get()); + assertEquals(freeT, handlerCounters.get(Free).getTotal().get()); + assertEquals(freeF, handlerCounters.get(Free).getFailure().get()); assertEquals(initT, handlerCounters.get(Init).getTotal().get()); assertEquals(initF, handlerCounters.get(Init).getFailure().get()); - assertEquals(disabledT, handlerCounters.get(Deleted).getTotal().get()); - assertEquals(disabledF, handlerCounters.get(Deleted).getFailure().get()); + assertEquals(deletedT, handlerCounters.get(Deleted).getTotal().get()); + assertEquals(deletedF, handlerCounters.get(Deleted).getFailure().get()); }); } @@ -1215,7 +1214,7 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, long released, long splitting, long free, - long disabled, + long deleted, long init ) throws IllegalAccessException { @@ -1231,8 +1230,8 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, assertEquals(owned, ownerLookUpCounters.get(Owned).get()); assertEquals(released, ownerLookUpCounters.get(Released).get()); assertEquals(splitting, ownerLookUpCounters.get(Splitting).get()); - assertEquals(free, ownerLookUpCounters.get(Disabled).get()); - assertEquals(disabled, ownerLookUpCounters.get(Deleted).get()); + assertEquals(free, ownerLookUpCounters.get(Free).get()); + assertEquals(deleted, ownerLookUpCounters.get(Deleted).get()); assertEquals(init, ownerLookUpCounters.get(Init).get()); }); } 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/ServiceUnitStateCompactionStrategyTest.java index cfdfe7c2c71a4..44335e00f98f4 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/ServiceUnitStateCompactionStrategyTest.java @@ -20,7 +20,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; @@ -50,23 +49,15 @@ public void test() throws InterruptedException { String dst = "dst"; String src = "src"; assertFalse(strategy.shouldKeepLeft(data(Init), data(Init))); - assertFalse(strategy.shouldKeepLeft(data(Init), data(Disabled))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Free))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigned))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Owned))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Released))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Splitting))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Deleted))); - assertFalse(strategy.shouldKeepLeft(data(Disabled), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Disabled))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Released))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Splitting))); - assertTrue(strategy.shouldKeepLeft(data(Disabled), data(Deleted))); - assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst1"), data(Owned, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, src, dst))); @@ -79,10 +70,7 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Owned), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Disabled, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Disabled, "dst2"))); - assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Disabled, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Disabled, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigned, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, src, dst))); @@ -90,6 +78,10 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst, "dst1"))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Owned))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Released, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Released, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Released, "dst2"))); + assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Released, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Released, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Splitting, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Splitting, "dst2"))); assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Splitting, dst))); @@ -97,7 +89,9 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Owned), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Released), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Disabled))); + assertFalse(strategy.shouldKeepLeft(data(Released), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Free, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Released, "src1", dst), data(Free, "src2", dst))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Owned, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Released, src, "dst1"), data(Owned, src, "dst2"))); @@ -108,7 +102,7 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Released), data(Deleted, dst))); assertFalse(strategy.shouldKeepLeft(data(Splitting), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Owned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Released))); @@ -120,7 +114,7 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Splitting, src, dst), data(Deleted, src, dst))); assertFalse(strategy.shouldKeepLeft(data(Deleted), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Owned))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Released))); @@ -128,7 +122,7 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Deleted))); assertFalse(strategy.shouldKeepLeft(data(Free), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Disabled))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Free))); assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, src, dst))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned))); 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 288b683086c08..77b6c97508ade 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 @@ -20,7 +20,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Disabled; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; @@ -36,23 +36,23 @@ public class ServiceUnitStateTest { public void testTransitions() { assertTrue(ServiceUnitState.isValidTransition(Init, Init)); - assertTrue(ServiceUnitState.isValidTransition(Init, Disabled)); - assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); + assertTrue(ServiceUnitState.isValidTransition(Init, Free)); assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Init, Released)); assertTrue(ServiceUnitState.isValidTransition(Init, Splitting)); assertTrue(ServiceUnitState.isValidTransition(Init, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Disabled, Init)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Disabled)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Assigned)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Released)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Disabled, Deleted)); + assertTrue(ServiceUnitState.isValidTransition(Free, Init)); + assertFalse(ServiceUnitState.isValidTransition(Free, Free)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Free, Released)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Free, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Init)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Free)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Owned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Released)); @@ -60,15 +60,15 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Assigned, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Owned, Init)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Free)); assertTrue(ServiceUnitState.isValidTransition(Owned, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Released)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Released)); assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Released, Init)); - assertFalse(ServiceUnitState.isValidTransition(Released, Disabled)); + assertTrue(ServiceUnitState.isValidTransition(Released, Free)); assertFalse(ServiceUnitState.isValidTransition(Released, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Released, Owned)); assertFalse(ServiceUnitState.isValidTransition(Released, Released)); @@ -76,7 +76,7 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Released, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Splitting, Init)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Released)); @@ -84,7 +84,7 @@ public void testTransitions() { assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted)); assertTrue(ServiceUnitState.isValidTransition(Deleted, Init)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Disabled)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Free)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Released)); 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 568f8fd8e50c6..8906d8313d9fb 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 @@ -122,7 +122,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)) .collect(Collectors.toList()); if (candidates.size() == 0) { return Init; From e2290b3083d1e822230e5b01fc44a429c08955e0 Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Tue, 21 Feb 2023 18:53:02 -0800 Subject: [PATCH 5/7] updated monitor logic --- .../extensions/channel/ServiceUnitState.java | 12 +- .../channel/ServiceUnitStateChannelImpl.java | 290 ++++++++----- .../ServiceUnitStateCompactionStrategy.java | 14 +- .../channel/ServiceUnitStateData.java | 15 +- .../ExtensibleLoadManagerImplTest.java | 31 +- .../channel/ServiceUnitStateChannelTest.java | 380 ++++++++++++++---- ...erviceUnitStateCompactionStrategyTest.java | 18 +- .../channel/ServiceUnitStateTest.java | 10 +- .../ServiceUnitStateCompactionTest.java | 24 +- 9 files changed, 574 insertions(+), 220 deletions(-) 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 40dd87c85d711..43a3b0a3e325b 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,15 +42,15 @@ public enum ServiceUnitState { Deleted; // deleted in the system (semi-terminal state) - private static Map> validTransitions = Map.of( + private static final Map> validTransitions = 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, Assigned, Released, Splitting, Deleted, Init), + Init, Set.of(Free, Owned, Assigned, Released, Splitting, Deleted), Free, Set.of(Assigned, Init), - Owned, Set.of(Assigned, Splitting, Released, Init), - Assigned, Set.of(Owned, Released, Init), - Released, Set.of(Owned, Free, Init), - Splitting, Set.of(Deleted, Init), + Owned, Set.of(Assigned, Splitting, Released), + Assigned, Set.of(Owned, Released), + Released, Set.of(Owned, Free), + Splitting, Set.of(Deleted), Deleted, Set.of(Init) ); 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 9ff272c1e1f76..f1656bbadef5a 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 @@ -38,6 +38,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Jittery; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Stable; 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.metadata.api.extended.SessionEvent.SessionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionReestablished; import com.google.common.annotations.VisibleForTesting; @@ -68,8 +69,11 @@ import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistry; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper; +import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; +import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -103,7 +107,6 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private static final long OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS = 60; 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 final PulsarService pulsar; @@ -113,6 +116,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final String lookupServiceAddress; private final ConcurrentOpenHashMap> cleanupJobs; private final LeaderElectionService leaderElectionService; + private BrokerSelectionStrategy brokerSelector; private BrokerRegistry brokerRegistry; private TableView tableview; private Producer producer; @@ -126,17 +130,16 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private long maxCleanupDelayTimeInSecs; private long minCleanupDelayTimeInSecs; // cleanup metrics - private long totalCleanupCnt = 0; - private long totalBrokerCleanupTombstoneCnt = 0; - private long totalServiceUnitCleanupTombstoneCnt = 0; + private long totalInactiveBrokerCleanupCnt = 0; + private long totalServiceUnitTombstoneCleanupCnt = 0; + + private long totalOrphanServiceUnitCleanupCnt = 0; private AtomicLong totalCleanupErrorCnt = new AtomicLong(); - private long totalCleanupScheduledCnt = 0; - private long totalCleanupIgnoredCnt = 0; - private long totalCleanupCancelledCnt = 0; + private long totalInactiveBrokerCleanupScheduledCnt = 0; + private long totalInactiveBrokerCleanupIgnoredCnt = 0; + private long totalInactiveBrokerCleanupCancelledCnt = 0; private volatile ChannelState channelState; - private AtomicLong totalBundleSplitRetryCount = new AtomicLong(); - public enum EventType { Assign, Split, @@ -246,14 +249,13 @@ public synchronized void start() throws PulsarServerException { boolean debug = debug(); try { this.brokerRegistry = getBrokerRegistry(); - this.brokerRegistry.addListener((broker, type) -> { - handleBrokerRegistrationEvent(broker, type); - }); + this.brokerRegistry.addListener(this::handleBrokerRegistrationEvent); leaderElectionService.start(); this.channelState = LeaderElectionServiceStarted; if (debug) { log.info("Successfully started the channel leader election service."); } + brokerSelector = getBrokerSelector(); if (producer != null) { producer.close(); @@ -308,6 +310,18 @@ protected BrokerRegistry getBrokerRegistry() { .get().getBrokerRegistry(); } + @VisibleForTesting + protected LoadManagerContext getContext() { + return ((ExtensibleLoadManagerWrapper) pulsar.getLoadManager().get()) + .get().getContext(); + } + + @VisibleForTesting + protected BrokerSelectionStrategy getBrokerSelector() { + // TODO: make this selector configurable. + return new LeastResourceUsageWithWeight(); + } + public synchronized void close() throws PulsarServerException { channelState = Closed; boolean debug = debug(); @@ -413,14 +427,15 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } ServiceUnitStateData data = tableview.get(serviceUnit); - ServiceUnitState state = data == null ? Init : data.state(); + ServiceUnitState state = state(data); ownerLookUpCounters.get(state).incrementAndGet(); switch (state) { case Owned, Splitting -> { return CompletableFuture.completedFuture(Optional.of(data.broker())); } case Assigned, Released -> { - return deferGetOwnerRequest(serviceUnit).thenApply(Optional::of); + return deferGetOwnerRequest(serviceUnit).thenApply( + broker -> broker == null ? Optional.empty() : Optional.of(broker)); } case Init, Free -> { return CompletableFuture.completedFuture(Optional.empty()); @@ -492,14 +507,16 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { lookupServiceAddress, serviceUnit, data, totalHandledRequests); } - ServiceUnitState state = data == null ? Init : data.state(); + ServiceUnitState state = state(data); try { switch (state) { case Owned -> handleOwnEvent(serviceUnit, data); case Assigned -> handleAssignEvent(serviceUnit, data); case Released -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); - case Deleted, Free, Init -> handleInitEvent(serviceUnit); + case Deleted -> handleDeleteEvent(serviceUnit, data); + case Free -> handleFreeEvent(serviceUnit, data); + case Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } } catch (Throwable e){ @@ -534,7 +551,7 @@ private AtomicLong getHandlerFailureCounter(ServiceUnitStateData data) { } private AtomicLong getHandlerCounter(ServiceUnitStateData data, boolean total) { - var state = data == null ? Init : data.state(); + var state = state(data); var counter = total ? handlerCounters.get(state).getTotal() : handlerCounters.get(state).getFailure(); if (counter == null) { @@ -580,7 +597,6 @@ private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) { } private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) { - deferGetOwnerRequest(serviceUnit); if (isTargetBroker(data.broker())) { ServiceUnitStateData next = new ServiceUnitStateData( isTransferCommand(data) ? Released : Owned, data.broker(), data.sourceBroker()); @@ -590,7 +606,6 @@ private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) { } private void handleReleaseEvent(String serviceUnit, ServiceUnitStateData data) { - if (isTransferCommand(data)) { if (isTargetBroker(data.sourceBroker())) { ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker(), data.sourceBroker()); @@ -616,24 +631,32 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { } } - private void handleDisableEvent(String serviceUnit, ServiceUnitStateData data) { + private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { + var getOwnerRequest = getOwnerRequests.remove(serviceUnit); + if (getOwnerRequest != null) { + getOwnerRequest.complete(null); + } if (isTargetBroker(data.broker())) { - ServiceUnitStateData next = new ServiceUnitStateData(Free, data.broker()); - pubAsync(serviceUnit, next) - .whenComplete((__, e) -> log(e, serviceUnit, data, next)); + log(null, serviceUnit, data, null); + } + } + + private void handleDeleteEvent(String serviceUnit, ServiceUnitStateData data) { + var getOwnerRequest = getOwnerRequests.remove(serviceUnit); + if (getOwnerRequest != null) { + getOwnerRequest.completeExceptionally(new IllegalStateException(serviceUnit + "has been deleted.")); + } + if (isTargetBroker(data.broker())) { + log(null, serviceUnit, data, null); } } private void handleInitEvent(String serviceUnit) { - closeServiceUnit(serviceUnit) - .thenAccept(__ -> { - var request = getOwnerRequests.remove(serviceUnit); - if (request != null) { - request.completeExceptionally(new IllegalStateException("The ownership has been unloaded. " - + "No owner is found for serviceUnit: " + serviceUnit)); - } - }) - .whenComplete((__, e) -> log(e, serviceUnit, null, null)); + var getOwnerRequest = getOwnerRequests.remove(serviceUnit); + if (getOwnerRequest != null) { + getOwnerRequest.complete(null); + } + log(null, serviceUnit, null, null); } private CompletableFuture pubAsync(String serviceUnit, ServiceUnitStateData data) { @@ -744,10 +767,6 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, CompletableFuture completionFuture) { CompletableFuture> updateFuture = new CompletableFuture<>(); - if (counter.get() > 0) { - totalBundleSplitRetryCount.incrementAndGet(); - } - pulsar.getNamespaceService().getSplitBoundary(bundle, null).thenAccept(splitBundlesPair -> { // Split and updateNamespaceBundles. Update may fail because of concurrent write to Zookeeper. if (splitBundlesPair == null) { @@ -863,7 +882,7 @@ private void handleBrokerCreationEvent(String broker) { CompletableFuture future = cleanupJobs.remove(broker); if (future != null) { future.cancel(false); - totalCleanupCancelledCnt++; + totalInactiveBrokerCleanupCancelledCnt++; log.info("Successfully cancelled the ownership cleanup for broker:{}." + " Active cleanup job count:{}", broker, cleanupJobs.size()); @@ -886,7 +905,7 @@ private void handleBrokerDeletionEvent(String broker) { case Stable -> scheduleCleanup(broker, minCleanupDelayTimeInSecs); case Jittery -> scheduleCleanup(broker, maxCleanupDelayTimeInSecs); case Unstable -> { - totalCleanupIgnoredCnt++; + totalInactiveBrokerCleanupIgnoredCnt++; log.error("MetadataState state is unstable. " + "Ignoring the ownership cleanup request for the reported broker :{}", broker); } @@ -897,7 +916,7 @@ private void scheduleCleanup(String broker, long delayInSecs) { cleanupJobs.computeIfAbsent(broker, k -> { Executor delayed = CompletableFuture .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); - totalCleanupScheduledCnt++; + totalInactiveBrokerCleanupScheduledCnt++; return CompletableFuture .runAsync(() -> { try { @@ -915,27 +934,40 @@ private void scheduleCleanup(String broker, long delayInSecs) { broker, delayInSecs, cleanupJobs.size()); } + private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, Set availableBrokers) { + + Optional selectedBroker = brokerSelector.select(availableBrokers, null, getContext()); + if (selectedBroker.isPresent()) { + var override = new ServiceUnitStateData(Owned, selectedBroker.get(), true); + log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}", + serviceUnit, orphanData, override); + pubAsync(serviceUnit, override).whenComplete((__, e) -> { + if (e != null) { + log.error("Failed to override serviceUnit:{} from orphanData:{} to overrideData:{}", + serviceUnit, orphanData, override, e); + } + }); + } else { + log.error("Failed to override the ownership serviceUnit:{} orphanData:{}. Empty selected broker.", + serviceUnit, orphanData); + } + } + - private void doCleanup(String broker) { + private void doCleanup(String broker) throws ExecutionException, InterruptedException, TimeoutException { long startTime = System.nanoTime(); log.info("Started ownership cleanup for the inactive broker:{}", broker); - int serviceUnitTombstoneCnt = 0; + int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); - for (Map.Entry etr : tableview.entrySet()) { - ServiceUnitStateData stateData = etr.getValue(); - String serviceUnit = etr.getKey(); + var availableBrokers = new HashSet(brokerRegistry.getAvailableBrokersAsync() + .get(inFlightStateWaitingTimeInMillis, MILLISECONDS)); + for (var etr : tableview.entrySet()) { + var stateData = etr.getValue(); + var serviceUnit = etr.getKey(); if (StringUtils.equals(broker, stateData.broker()) || StringUtils.equals(broker, stateData.sourceBroker())) { - log.info("Cleaning ownership serviceUnit:{}, stateData:{}.", serviceUnit, stateData); - tombstoneAsync(serviceUnit).whenComplete((__, e) -> { - if (e != null) { - log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " - + "cleanupErrorCnt:{}.", - serviceUnit, stateData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); - } - }); - serviceUnitTombstoneCnt++; + overrideOwnership(serviceUnit, stateData, availableBrokers); + orphanServiceUnitCleanupCnt++; } } @@ -945,26 +977,49 @@ private void doCleanup(String broker) { log.error("Failed to flush the in-flight messages.", e); } - if (serviceUnitTombstoneCnt > 0) { - this.totalCleanupCnt++; - this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt; - this.totalBrokerCleanupTombstoneCnt++; + if (orphanServiceUnitCleanupCnt > 0) { + this.totalOrphanServiceUnitCleanupCnt += orphanServiceUnitCleanupCnt; + this.totalInactiveBrokerCleanupCnt++; } double cleanupTime = TimeUnit.NANOSECONDS .toMillis((System.nanoTime() - startTime)); // TODO: clean load data stores log.info("Completed a cleanup for the inactive broker:{} in {} ms. " - + "Published tombstone for orphan service units: serviceUnitTombstoneCnt:{}, " + + "Cleaned up orphan service units: orphanServiceUnitCleanupCnt:{}, " + "approximate cleanupErrorCnt:{}, metrics:{} ", broker, cleanupTime, - serviceUnitTombstoneCnt, + orphanServiceUnitCleanupCnt, totalCleanupErrorCntStart - totalCleanupErrorCnt.get(), printCleanupMetrics()); cleanupJobs.remove(broker); } + private Optional getOverrideStateData(String serviceUnit, ServiceUnitStateData orphanData, + Set availableBrokers, + LoadManagerContext context) { + if (isTransferCommand(orphanData)) { + // rollback to the src + return Optional.of(new ServiceUnitStateData(Owned, orphanData.sourceBroker(), true)); + } else if (orphanData.state() == Assigned) { // assign + // roll-forward to another broker + Optional selectedBroker = brokerSelector.select(availableBrokers, null, context); + if (selectedBroker.isEmpty()) { + return Optional.empty(); + } + return Optional.of(new ServiceUnitStateData(Owned, selectedBroker.get(), true)); + } else if (orphanData.state() == Splitting || orphanData.state() == Released) { + // rollback to the target broker for split and unload + return Optional.of(new ServiceUnitStateData(Owned, orphanData.broker(), true)); + } else { + var msg = String.format("Failed to get the overrideStateData from serviceUnit=%s, orphanData=%s", + serviceUnit, orphanData); + log.error(msg); + throw new IllegalStateException(msg); + } + } + @VisibleForTesting protected void monitorOwnerships(List brokers) { if (!isChannelOwner()) { @@ -980,7 +1035,9 @@ protected void monitorOwnerships(List brokers) { long startTime = System.nanoTime(); Set inactiveBrokers = new HashSet<>(); Set activeBrokers = new HashSet<>(brokers); - int serviceUnitTombstoneCnt = 0; + Map orphanServiceUnits = new HashMap<>(); + int serviceUnitTombstoneCleanupCnt = 0; + int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); long now = System.currentTimeMillis(); for (Map.Entry etr : tableview.entrySet()) { @@ -992,59 +1049,82 @@ protected void monitorOwnerships(List brokers) { inactiveBrokers.add(stateData.broker()); } else if (state != Owned && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { - boolean tombstone = false; if (state == Deleted || state == Free) { if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { - log.info("Found semi-terminal states to clean" + log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); - tombstone = true; + tombstoneAsync(serviceUnit).whenComplete((__, e) -> { + if (e != null) { + log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " + + "cleanupErrorCnt:{}.", + serviceUnit, stateData, + totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); + } + }); + serviceUnitTombstoneCleanupCnt++; } } else { - log.warn("Found long-running orphan(in-flight) serviceUnit:{}, stateData:{}", - serviceUnit, stateData); - tombstone = true; + log.warn("Found orphan serviceUnit:{}, stateData:{}", serviceUnit, stateData); + orphanServiceUnits.put(serviceUnit, stateData); } + } + } - - if (tombstone) { - tombstoneAsync(serviceUnit).whenComplete((__, e) -> { + // Skip cleaning orphan bundles if inactiveBrokers exist. This is a bigger problem. + if (!inactiveBrokers.isEmpty()) { + for (String inactiveBroker : inactiveBrokers) { + handleBrokerDeletionEvent(inactiveBroker); + } + } else if (!orphanServiceUnits.isEmpty()) { + var context = getContext(); + for (var etr : orphanServiceUnits.entrySet()) { + var orphanServiceUnit = etr.getKey(); + var orphanData = etr.getValue(); + var overrideData = getOverrideStateData( + orphanServiceUnit, orphanData, activeBrokers, context); + if (overrideData.isPresent()) { + pubAsync(orphanServiceUnit, overrideData.get()).whenComplete((__, e) -> { if (e != null) { - log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, " + log.error("Failed cleaning the ownership orphanServiceUnit:{}, orphanData:{}, " + "cleanupErrorCnt:{}.", - serviceUnit, stateData, + orphanServiceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); } }); - serviceUnitTombstoneCnt++; + orphanServiceUnitCleanupCnt++; + } else { + log.warn("Failed get the overrideStateData from orphanServiceUnit:{}, orphanData:{}. will retry..", + orphanServiceUnit, orphanData); } } } - for (String inactiveBroker : inactiveBrokers) { - handleBrokerDeletionEvent(inactiveBroker); - } - try { producer.flush(); } catch (PulsarClientException e) { log.error("Failed to flush the in-flight messages.", e); } - if (serviceUnitTombstoneCnt > 0) { - this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt; + if (serviceUnitTombstoneCleanupCnt > 0) { + this.totalServiceUnitTombstoneCleanupCnt += serviceUnitTombstoneCleanupCnt; + } + + if (orphanServiceUnitCleanupCnt > 0) { + this.totalOrphanServiceUnitCleanupCnt += orphanServiceUnitCleanupCnt; } double monitorTime = TimeUnit.NANOSECONDS .toMillis((System.nanoTime() - startTime)); log.info("Completed the ownership monitor run in {} ms. " - + "Scheduled cleanups for inactiveBrokers:{}. inactiveBrokerCount:{}. " - + "Published tombstone for orphan service units: serviceUnitTombstoneCnt:{}, " - + "approximate cleanupErrorCnt:{}, metrics:{} ", + + "Scheduled cleanups for inactive brokers:{}. inactiveBrokerCount:{}. " + + "Published cleanups for orphan service units, orphanServiceUnitCleanupCnt:{}. " + + "Tombstoned semi-terminal state service units, serviceUnitTombstoneCleanupCnt:{}. " + + "Approximate cleanupErrorCnt:{}, metrics:{}. ", monitorTime, - inactiveBrokers, - inactiveBrokers.size(), - serviceUnitTombstoneCnt, + inactiveBrokers, inactiveBrokers.size(), + orphanServiceUnitCleanupCnt, + serviceUnitTombstoneCleanupCnt, totalCleanupErrorCntStart - totalCleanupErrorCnt.get(), printCleanupMetrics()); @@ -1052,17 +1132,19 @@ protected void monitorOwnerships(List brokers) { private String printCleanupMetrics() { return String.format( - "{totalCleanupCnt:%d, totalBrokerCleanupTombstoneCnt:%d, " - + "totalServiceUnitCleanupTombstoneCnt:%d, totalCleanupErrorCnt:%d, " - + "totalCleanupScheduledCnt%d, totalCleanupIgnoredCnt:%d, totalCleanupCancelledCnt:%d, " + "{totalInactiveBrokerCleanupCnt:%d, " + + "totalServiceUnitTombstoneCleanupCnt:%d, totalOrphanServiceUnitCleanupCnt:%d, " + + "totalCleanupErrorCnt:%d, " + + "totalInactiveBrokerCleanupScheduledCnt%d, totalInactiveBrokerCleanupIgnoredCnt:%d, " + + "totalInactiveBrokerCleanupCancelledCnt:%d, " + " activeCleanupJobs:%d}", - totalCleanupCnt, - totalBrokerCleanupTombstoneCnt, - totalServiceUnitCleanupTombstoneCnt, + totalInactiveBrokerCleanupCnt, + totalServiceUnitTombstoneCleanupCnt, + totalOrphanServiceUnitCleanupCnt, totalCleanupErrorCnt.get(), - totalCleanupScheduledCnt, - totalCleanupIgnoredCnt, - totalCleanupCancelledCnt, + totalInactiveBrokerCleanupScheduledCnt, + totalInactiveBrokerCleanupIgnoredCnt, + totalInactiveBrokerCleanupCancelledCnt, cleanupJobs.size() ); } @@ -1127,15 +1209,6 @@ public List getMetrics() { } } - - { - var dim = new HashMap<>(dimensions); - dim.put("result", "Total"); - var metric = Metrics.create(dim); - metric.put("brk_sunit_state_chn_cleanup_ops_total", totalCleanupCnt); - metrics.add(metric); - } - { var dim = new HashMap<>(dimensions); dim.put("result", "Failure"); @@ -1148,7 +1221,7 @@ public List getMetrics() { var dim = new HashMap<>(dimensions); dim.put("result", "Skip"); var metric = Metrics.create(dim); - metric.put("brk_sunit_state_chn_cleanup_ops_total", totalCleanupIgnoredCnt); + metric.put("brk_sunit_state_chn_inactive_broker_cleanup_ops_total", totalInactiveBrokerCleanupIgnoredCnt); metrics.add(metric); } @@ -1156,7 +1229,7 @@ public List getMetrics() { var dim = new HashMap<>(dimensions); dim.put("result", "Cancel"); var metric = Metrics.create(dim); - metric.put("brk_sunit_state_chn_cleanup_ops_total", totalCleanupCancelledCnt); + metric.put("brk_sunit_state_chn_inactive_broker_cleanup_ops_total", totalInactiveBrokerCleanupCancelledCnt); metrics.add(metric); } @@ -1164,13 +1237,14 @@ public List getMetrics() { var dim = new HashMap<>(dimensions); dim.put("result", "Schedule"); var metric = Metrics.create(dim); - metric.put("brk_sunit_state_chn_cleanup_ops_total", totalCleanupScheduledCnt); + metric.put("brk_sunit_state_chn_inactive_broker_cleanup_ops_total", totalInactiveBrokerCleanupScheduledCnt); metrics.add(metric); } var metric = Metrics.create(dimensions); - metric.put("brk_sunit_state_chn_broker_cleanup_ops_total", totalBrokerCleanupTombstoneCnt); - metric.put("brk_sunit_state_chn_su_cleanup_ops_total", totalServiceUnitCleanupTombstoneCnt); + metric.put("brk_sunit_state_chn_inactive_broker_cleanup_ops_total", totalInactiveBrokerCleanupCnt); + metric.put("brk_sunit_state_chn_orphan_su_cleanup_ops_total", totalOrphanServiceUnitCleanupCnt); + metric.put("brk_sunit_state_chn_su_tombstone_cleanup_ops_total", totalServiceUnitTombstoneCleanupCnt); metrics.add(metric); return metrics; 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/ServiceUnitStateCompactionStrategy.java index 0833569cf5f1a..1c3196df97c66 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/ServiceUnitStateCompactionStrategy.java @@ -20,7 +20,7 @@ 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.Init; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Schema; @@ -48,8 +48,16 @@ public void checkBrokers(boolean check) { @Override public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to) { - ServiceUnitState prevState = from == null ? Init : from.state(); - ServiceUnitState state = to == null ? Init : to.state(); + if (to == null) { + return false; + } else if (to.force()) { + return false; + } + + + ServiceUnitState prevState = state(from); + ServiceUnitState state = state(to); + if (!ServiceUnitState.isValidTransition(prevState, state)) { return true; } 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 6a7bf97b46b88..6a04431de64d5 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 @@ -27,7 +27,8 @@ * This data will be broadcast in ServiceUnitStateChannel. */ -public record ServiceUnitStateData(ServiceUnitState state, String broker, String sourceBroker, long timestamp) { +public record ServiceUnitStateData( + ServiceUnitState state, String broker, String sourceBroker, boolean force, long timestamp) { public ServiceUnitStateData { Objects.requireNonNull(state); @@ -37,10 +38,18 @@ public record ServiceUnitStateData(ServiceUnitState state, String broker, String } public ServiceUnitStateData(ServiceUnitState state, String broker, String sourceBroker) { - this(state, broker, sourceBroker, System.currentTimeMillis()); + this(state, broker, sourceBroker, false, System.currentTimeMillis()); } public ServiceUnitStateData(ServiceUnitState state, String broker) { - this(state, broker, null, System.currentTimeMillis()); + this(state, broker, null, false, System.currentTimeMillis()); + } + + public ServiceUnitStateData(ServiceUnitState state, String broker, boolean force) { + this(state, broker, null, force, System.currentTimeMillis()); + } + + public static ServiceUnitState state(ServiceUnitStateData data) { + return data == null ? ServiceUnitState.Init : data.state(); } } 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 8d386371b6bf4..39756006b24bb 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 @@ -392,13 +392,15 @@ SplitDecision.Reason.Balanced, new MutableLong(6) } { - FieldUtils.writeDeclaredField(channel1, "totalCleanupCnt", 1, true); - FieldUtils.writeDeclaredField(channel1, "totalBrokerCleanupTombstoneCnt", 2, true); - FieldUtils.writeDeclaredField(channel1, "totalServiceUnitCleanupTombstoneCnt", 3, true); + + FieldUtils.writeDeclaredField(channel1, "totalInactiveBrokerCleanupCnt", 1, true); + FieldUtils.writeDeclaredField(channel1, "totalServiceUnitTombstoneCleanupCnt", 2, true); + FieldUtils.writeDeclaredField(channel1, "totalOrphanServiceUnitCleanupCnt", 3, true); FieldUtils.writeDeclaredField(channel1, "totalCleanupErrorCnt", new AtomicLong(4), true); - FieldUtils.writeDeclaredField(channel1, "totalCleanupScheduledCnt", 5, true); - FieldUtils.writeDeclaredField(channel1, "totalCleanupIgnoredCnt", 6, true); - FieldUtils.writeDeclaredField(channel1, "totalCleanupCancelledCnt", 7, true); + FieldUtils.writeDeclaredField(channel1, "totalInactiveBrokerCleanupScheduledCnt", 5, true); + FieldUtils.writeDeclaredField(channel1, "totalInactiveBrokerCleanupIgnoredCnt", 6, true); + FieldUtils.writeDeclaredField(channel1, "totalInactiveBrokerCleanupCancelledCnt", 7, true); + Map ownerLookUpCounters = new LinkedHashMap<>(); Map handlerCounters = new LinkedHashMap<>(); Map eventCounters = @@ -431,9 +433,6 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] @@ -441,6 +440,9 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoLoadData, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=7}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBrokers, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=8}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] @@ -451,9 +453,9 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundlesSplit, reason=Admin, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=5}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] + dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] @@ -481,12 +483,11 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=14}] - dimensions=[{broker=localhost, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=4}] - dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=6}] - dimensions=[{broker=localhost, metric=sunitStateChn, result=Cancel}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=7}] - dimensions=[{broker=localhost, metric=sunitStateChn, result=Schedule}], metrics=[{brk_sunit_state_chn_cleanup_ops_total=5}] - dimensions=[{broker=localhost, metric=sunitStateChn}], metrics=[{brk_sunit_state_chn_broker_cleanup_ops_total=2, brk_sunit_state_chn_su_cleanup_ops_total=3}] + dimensions=[{broker=localhost, metric=sunitStateChn, result=Skip}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=6}] + dimensions=[{broker=localhost, metric=sunitStateChn, result=Cancel}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=7}] + dimensions=[{broker=localhost, metric=sunitStateChn, result=Schedule}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=5}] + dimensions=[{broker=localhost, metric=sunitStateChn}], metrics=[{brk_sunit_state_chn_inactive_broker_cleanup_ops_total=1, brk_sunit_state_chn_orphan_su_cleanup_ops_total=3, brk_sunit_state_chn_su_tombstone_cleanup_ops_total=2}] """.split("\n")); var actual = primaryLoadManager.getMetrics().stream().map(m -> m.toString()).collect(Collectors.toSet()); assertEquals(actual, expected); 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 663c6e295c87d..255488f0bbc78 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 @@ -29,6 +29,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Split; 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.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; @@ -75,8 +76,10 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistryImpl; +import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; +import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.Producer; @@ -108,8 +111,12 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private String bundle2; private PulsarTestContext additionalPulsarTestContext; + private LoadManagerContext loadManagerContext; + private BrokerRegistryImpl registry; + private BrokerSelectionStrategy brokerSelector; + @BeforeClass @Override protected void setup() throws Exception { @@ -124,10 +131,11 @@ protected void setup() throws Exception { pulsar1 = pulsar; registry = new BrokerRegistryImpl(pulsar); + loadManagerContext = mock(LoadManagerContext.class); + brokerSelector = mock(BrokerSelectionStrategy.class); additionalPulsarTestContext = createAdditionalPulsarTestContext(getDefaultConf()); pulsar2 = additionalPulsarTestContext.getPulsarService(); - channel1 = createChannel(pulsar1); channel1.start(); @@ -429,8 +437,7 @@ public void transferTest() @Test(priority = 5) public void transferTestWhenDestBrokerFails() - throws ExecutionException, InterruptedException, IllegalAccessException, PulsarServerException, - TimeoutException { + throws ExecutionException, InterruptedException, IllegalAccessException { var getOwnerRequests1 = getOwnerRequests(channel1); var getOwnerRequests2 = getOwnerRequests(channel2); @@ -483,9 +490,8 @@ public void transferTestWhenDestBrokerFails() assertEquals(0, getOwnerRequests1.size()); assertEquals(0, getOwnerRequests2.size()); - // recovered, check the monitor update state : Assigned -> Init + // recovered, check the monitor update state : Assigned -> Owned FieldUtils.writeDeclaredField(channel2, "producer", producer, true); - FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel2, @@ -495,8 +501,25 @@ public void transferTestWhenDestBrokerFails() List.of(lookupServiceAddress1, lookupServiceAddress2)); ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( List.of(lookupServiceAddress1, lookupServiceAddress2)); - waitUntilState(channel1, bundle, Init); - waitUntilState(channel2, bundle, Init); + + + waitUntilNewOwner(channel1, bundle, lookupServiceAddress1); + waitUntilNewOwner(channel2, bundle, lookupServiceAddress1); + ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(lookupServiceAddress1)); + + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 0, + 1, + 0, + 0, + 0, + 0); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); @@ -559,7 +582,7 @@ public void splitAndRetryTest() throws Exception { assertEquals(Optional.of(lookupServiceAddress1), channel2.getOwnerAsync(childBundle2).get()); - // try the monitor and check the monitor moves `Disabled` -> `Init` + // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); FieldUtils.writeDeclaredField(channel1, @@ -577,6 +600,16 @@ public void splitAndRetryTest() throws Exception { waitUntilState(channel1, bundle, Init); waitUntilState(channel2, bundle, Init); + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 1, + 0, + 0, + 0, + 0, + 0); + cleanTableView(channel1, childBundle1); cleanTableView(channel2, childBundle1); cleanTableView(channel1, childBundle2); @@ -671,6 +704,7 @@ public void handleBrokerDeletionEventTest() var owner1 = channel1.getOwnerAsync(bundle1); var owner2 = channel2.getOwnerAsync(bundle2); + doReturn(Optional.of(lookupServiceAddress2)).when(brokerSelector).select(any(), any(), any()); assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); @@ -693,26 +727,30 @@ public void handleBrokerDeletionEventTest() leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); - waitUntilNewOwner(channel1, bundle1, null); - waitUntilNewOwner(channel2, bundle1, null); - waitUntilNewOwner(channel1, bundle2, null); - waitUntilNewOwner(channel2, bundle2, null); + waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle1, lookupServiceAddress2); + waitUntilNewOwner(channel1, bundle2, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle2, lookupServiceAddress2); verify(leaderCleanupJobs, times(1)).computeIfAbsent(eq(broker), any()); verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); + + assertEquals(0, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 1, + 0, + 2, + 0, + 1, + 0, + 0); + // test jittery metadata state - channel1.publishAssignEventAsync(bundle1, broker); - channel2.publishAssignEventAsync(bundle2, broker); + channel1.publishUnloadEventAsync(new Unload(lookupServiceAddress2, bundle1, Optional.of(broker))); + channel1.publishUnloadEventAsync(new Unload(lookupServiceAddress2, bundle2, Optional.of(broker))); waitUntilNewOwner(channel1, bundle1, broker); waitUntilNewOwner(channel2, bundle1, broker); waitUntilNewOwner(channel1, bundle2, broker); @@ -727,13 +765,14 @@ public void handleBrokerDeletionEventTest() verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); assertEquals(1, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 1, + 0, + 2, + 0, + 2, + 0, + 0); // broker is back online leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Created); @@ -743,13 +782,14 @@ public void handleBrokerDeletionEventTest() verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); assertEquals(0, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 1, + 0, + 2, + 0, + 2, + 0, + 1); // broker is offline again @@ -761,35 +801,37 @@ public void handleBrokerDeletionEventTest() verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); assertEquals(1, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(3, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 1, + 0, + 2, + 0, + 3, + 0, + 1); // finally cleanup - waitUntilNewOwner(channel1, bundle1, null); - waitUntilNewOwner(channel2, bundle1, null); - waitUntilNewOwner(channel1, bundle2, null); - waitUntilNewOwner(channel2, bundle2, null); + waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle1, lookupServiceAddress2); + waitUntilNewOwner(channel1, bundle2, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle2, lookupServiceAddress2); verify(leaderCleanupJobs, times(3)).computeIfAbsent(eq(broker), any()); verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); assertEquals(0, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(2, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(4, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(3, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 2, + 0, + 4, + 0, + 3, + 0, + 1); // test unstable state - channel1.publishAssignEventAsync(bundle1, broker); - channel2.publishAssignEventAsync(bundle2, broker); + channel1.publishUnloadEventAsync(new Unload(lookupServiceAddress2, bundle1, Optional.of(broker))); + channel1.publishUnloadEventAsync(new Unload(lookupServiceAddress2, bundle2, Optional.of(broker))); waitUntilNewOwner(channel1, bundle1, broker); waitUntilNewOwner(channel2, bundle1, broker); waitUntilNewOwner(channel1, bundle2, broker); @@ -804,13 +846,14 @@ public void handleBrokerDeletionEventTest() verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); assertEquals(0, leaderCleanupJobs.size()); assertEquals(0, followerCleanupJobs.size()); - assertEquals(2, getCleanupMetric(leaderChannel, "totalCleanupCnt")); - assertEquals(2, getCleanupMetric(leaderChannel, "totalBrokerCleanupTombstoneCnt")); - assertEquals(4, getCleanupMetric(leaderChannel, "totalServiceUnitCleanupTombstoneCnt")); - assertEquals(0, getCleanupMetric(leaderChannel, "totalCleanupErrorCnt")); - assertEquals(3, getCleanupMetric(leaderChannel, "totalCleanupScheduledCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupIgnoredCnt")); - assertEquals(1, getCleanupMetric(leaderChannel, "totalCleanupCancelledCnt")); + validateMonitorCounters(leaderChannel, + 2, + 0, + 4, + 0, + 3, + 1, + 1); // clean-up FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3 * 60, true); @@ -969,6 +1012,17 @@ public void unloadTest() waitUntilState(channel1, bundle, Init); waitUntilState(channel2, bundle, Init); + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 1, + 0, + 0, + 0, + 0, + 0); + + FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, @@ -980,6 +1034,169 @@ public void unloadTest() "semiTerminalStateWaitingTimeInMillis", 300 * 1000, true); } + @Test(priority = 13) + public void assignTestWhenDestBrokerFails() + throws ExecutionException, InterruptedException, IllegalAccessException { + + Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.empty()); + + channel1.publishUnloadEventAsync(unload); + + waitUntilState(channel1, bundle, Free); + waitUntilState(channel2, bundle, Free); + + 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 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); + doReturn(Optional.of(lookupServiceAddress2)).when(brokerSelector).select(any(), any(), any()); + channel1.publishAssignEventAsync(bundle, lookupServiceAddress2); + // channel1 is broken. the assign won't be complete. + waitUntilState(channel1, bundle); + waitUntilState(channel2, bundle); + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertFalse(owner1.isDone()); + assertFalse(owner2.isDone()); + + // In 5 secs, the getOwnerAsync requests(lookup requests) should time out. + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); + + // recovered, check the monitor update state : Assigned -> Owned + FieldUtils.writeDeclaredField(channel2, "producer", producer, true); + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + + + waitUntilNewOwner(channel1, bundle, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle, lookupServiceAddress2); + var ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + var ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(lookupServiceAddress2)); + + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 0, + 1, + 0, + 0, + 0, + 0); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + + } + + @Test(priority = 14) + public void splitTestWhenDestBrokerFails() + throws ExecutionException, InterruptedException, IllegalAccessException { + + + Unload unload = new Unload(lookupServiceAddress1, bundle, Optional.empty()); + + channel1.publishUnloadEventAsync(unload); + + waitUntilState(channel1, bundle, Free); + waitUntilState(channel2, bundle, Free); + + channel1.publishAssignEventAsync(bundle, lookupServiceAddress1); + + waitUntilState(channel1, bundle, Owned); + waitUntilState(channel2, bundle, Owned); + + assertEquals(lookupServiceAddress1, channel1.getOwnerAsync(bundle).get().get()); + assertEquals(lookupServiceAddress1, channel2.getOwnerAsync(bundle).get().get()); + + var producer = (Producer) FieldUtils.readDeclaredField(channel1, + "producer", true); + var spyProducer = spy(producer); + var msg = mock(TypedMessageBuilder.class); + 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); + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + channel2.publishSplitEventAsync(new Split(bundle, lookupServiceAddress1, null)); + // channel1 is broken. the split won't be complete. + waitUntilState(channel1, bundle); + waitUntilState(channel2, bundle); + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + + // recovered, check the monitor update state : Splitting -> Owned + FieldUtils.writeDeclaredField(channel1, "producer", producer, true); + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1 , true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1 , true); + + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(lookupServiceAddress1, lookupServiceAddress2)); + + + waitUntilNewOwner(channel1, bundle, lookupServiceAddress1); + waitUntilNewOwner(channel2, bundle, lookupServiceAddress1); + var ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + var ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(lookupServiceAddress1)); + + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 0, + 1, + 0, + 0, + 0, + 0); + + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + + } + private static ConcurrentOpenHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { return (ConcurrentOpenHashMap>>) @@ -1076,7 +1293,7 @@ private static void waitUntilState(ServiceUnitStateChannel channel, String key, .atMost(10, TimeUnit.SECONDS) .until(() -> { // wait until true ServiceUnitStateData data = tv.get(key); - ServiceUnitState actual = data == null ? Init : data.state(); + ServiceUnitState actual = state(data); return actual == expected; }); } @@ -1133,13 +1350,13 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) } private void cleanOwnershipMonitorCounters(ServiceUnitStateChannel channel) throws IllegalAccessException { - FieldUtils.writeDeclaredField(channel, "totalCleanupCnt", 0, true); - FieldUtils.writeDeclaredField(channel, "totalBrokerCleanupTombstoneCnt", 0, true); - FieldUtils.writeDeclaredField(channel, "totalServiceUnitCleanupTombstoneCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalInactiveBrokerCleanupCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalServiceUnitTombstoneCleanupCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalOrphanServiceUnitCleanupCnt", 0, true); FieldUtils.writeDeclaredField(channel, "totalCleanupErrorCnt", new AtomicLong(0), true); - FieldUtils.writeDeclaredField(channel, "totalCleanupScheduledCnt", 0, true); - FieldUtils.writeDeclaredField(channel, "totalCleanupIgnoredCnt", 0, true); - FieldUtils.writeDeclaredField(channel, "totalCleanupCancelledCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalInactiveBrokerCleanupScheduledCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalInactiveBrokerCleanupIgnoredCnt", 0, true); + FieldUtils.writeDeclaredField(channel, "totalInactiveBrokerCleanupCancelledCnt", 0, true); } private static long getCleanupMetric(ServiceUnitStateChannel channel, String metric) @@ -1236,13 +1453,38 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, }); } + private static void validateMonitorCounters(ServiceUnitStateChannel channel, + long totalInactiveBrokerCleanupCnt, + long totalServiceUnitTombstoneCleanupCnt, + long totalOrphanServiceUnitCleanupCnt, + long totalCleanupErrorCnt, + long totalInactiveBrokerCleanupScheduledCnt, + long totalInactiveBrokerCleanupIgnoredCnt, + long totalInactiveBrokerCleanupCancelledCnt) + throws IllegalAccessException { + assertEquals(totalInactiveBrokerCleanupCnt, getCleanupMetric(channel, "totalInactiveBrokerCleanupCnt")); + assertEquals(totalServiceUnitTombstoneCleanupCnt, + getCleanupMetric(channel, "totalServiceUnitTombstoneCleanupCnt")); + assertEquals(totalOrphanServiceUnitCleanupCnt, getCleanupMetric(channel, "totalOrphanServiceUnitCleanupCnt")); + assertEquals(totalCleanupErrorCnt, getCleanupMetric(channel, "totalCleanupErrorCnt")); + assertEquals(totalInactiveBrokerCleanupScheduledCnt, + getCleanupMetric(channel, "totalInactiveBrokerCleanupScheduledCnt")); + assertEquals(totalInactiveBrokerCleanupIgnoredCnt, + getCleanupMetric(channel, "totalInactiveBrokerCleanupIgnoredCnt")); + assertEquals(totalInactiveBrokerCleanupCancelledCnt, + getCleanupMetric(channel, "totalInactiveBrokerCleanupCancelledCnt")); + } + ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) throws IllegalAccessException { var tmpChannel = new ServiceUnitStateChannelImpl(pulsar); FieldUtils.writeDeclaredField(tmpChannel, "ownershipMonitorDelayTimeInSecs", 5, true); var channel = spy(tmpChannel); + doReturn(loadManagerContext).when(channel).getContext(); doReturn(registry).when(channel).getBrokerRegistry(); + doReturn(brokerSelector).when(channel).getBrokerSelector(); + return channel; } } 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/ServiceUnitStateCompactionStrategyTest.java index 44335e00f98f4..4a5ac391a917c 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/ServiceUnitStateCompactionStrategyTest.java @@ -48,7 +48,15 @@ ServiceUnitStateData data(ServiceUnitState state, String src, String dst) { public void test() throws InterruptedException { String dst = "dst"; String src = "src"; - assertFalse(strategy.shouldKeepLeft(data(Init), data(Init))); + + assertFalse(strategy.shouldKeepLeft( + new ServiceUnitStateData(Init, dst), + new ServiceUnitStateData(Init, dst, true))); + + assertFalse(strategy.shouldKeepLeft( + data(Owned), null)); + + assertTrue(strategy.shouldKeepLeft(data(Init), data(Init))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Free))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigned))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Owned))); @@ -56,7 +64,7 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Init), data(Splitting))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Deleted))); - assertFalse(strategy.shouldKeepLeft(data(Assigned), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst1"), data(Owned, "dst2"))); @@ -69,7 +77,7 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Splitting, dst))); assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Deleted, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigned, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst))); @@ -88,7 +96,7 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Splitting, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Deleted, dst))); - assertFalse(strategy.shouldKeepLeft(data(Released), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Released), data(Init))); assertFalse(strategy.shouldKeepLeft(data(Released), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Free, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Released, "src1", dst), data(Free, "src2", dst))); @@ -101,7 +109,7 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Released), data(Splitting))); assertTrue(strategy.shouldKeepLeft(data(Released), data(Deleted, dst))); - assertFalse(strategy.shouldKeepLeft(data(Splitting), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Free))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigned))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Owned))); 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 77b6c97508ade..1a913cdbaabee 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 @@ -35,7 +35,7 @@ public class ServiceUnitStateTest { @Test public void testTransitions() { - assertTrue(ServiceUnitState.isValidTransition(Init, Init)); + assertFalse(ServiceUnitState.isValidTransition(Init, Init)); assertTrue(ServiceUnitState.isValidTransition(Init, Free)); assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); @@ -51,7 +51,7 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Free, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Assigned, Init)); + assertFalse(ServiceUnitState.isValidTransition(Assigned, Init)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Free)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Assigned, Owned)); @@ -59,7 +59,7 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Assigned, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Assigned, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Init)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Init)); assertFalse(ServiceUnitState.isValidTransition(Owned, Free)); assertTrue(ServiceUnitState.isValidTransition(Owned, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); @@ -67,7 +67,7 @@ public void testTransitions() { assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Released, Init)); + assertFalse(ServiceUnitState.isValidTransition(Released, Init)); assertTrue(ServiceUnitState.isValidTransition(Released, Free)); assertFalse(ServiceUnitState.isValidTransition(Released, Assigned)); assertTrue(ServiceUnitState.isValidTransition(Released, Owned)); @@ -75,7 +75,7 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Released, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Released, Deleted)); - assertTrue(ServiceUnitState.isValidTransition(Splitting, Init)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Init)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigned)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); 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 8906d8313d9fb..61299bc79dabf 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 @@ -25,6 +25,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isValidTransition; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; @@ -196,10 +197,21 @@ TestData generateTestData() throws PulsarAdminException, PulsarClientException { int keyIndex = r.nextInt(maxKeys); String key = "key" + keyIndex; ServiceUnitStateData prev = expected.get(key); - ServiceUnitState prevState = prev == null ? Init : prev.state(); - ServiceUnitState state = r.nextBoolean() ? nextInvalidState(prevState) : + ServiceUnitState prevState = state(prev); + boolean invalid = r.nextBoolean(); + ServiceUnitState state = invalid ? nextInvalidState(prevState) : nextValidState(prevState); - ServiceUnitStateData value = new ServiceUnitStateData(state, key + ":" + j); + ServiceUnitStateData value; + if (invalid) { + value = new ServiceUnitStateData(state, key + ":" + j, false); + } else { + if (state == Init) { + value = new ServiceUnitStateData(state, key + ":" + j, true); + } else { + value = new ServiceUnitStateData(state, key + ":" + j, false); + } + } + producer.newMessage().key(key).value(value).send(); if (!strategy.shouldKeepLeft(prev, value)) { expected.put(key, value); @@ -719,9 +731,9 @@ public void testCompactionWithLastDeletedKey() throws Exception { pulsarClient.newConsumer(schema).topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); - producer.newMessage().key("1").value(testValue(Owned, "1")).send(); - producer.newMessage().key("2").value(testValue(Owned, "3")).send(); - producer.newMessage().key("3").value(testValue(Owned, "5")).send(); + producer.newMessage().key("1").value(testValue("1")).send(); + producer.newMessage().key("2").value(testValue("3")).send(); + producer.newMessage().key("3").value(testValue( "5")).send(); producer.newMessage().key("1").value(null).send(); producer.newMessage().key("2").value(null).send(); From 2fbafc99356563dbfd8c067cc56a81f15599c07d Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Tue, 21 Feb 2023 22:25:21 -0800 Subject: [PATCH 6/7] Renamed in-flight states --- .../extensions/channel/ServiceUnitState.java | 24 +++-- .../channel/ServiceUnitStateChannelImpl.java | 38 ++++---- .../ServiceUnitStateCompactionStrategy.java | 12 +-- .../channel/ServiceUnitStateChannelTest.java | 36 ++++---- ...erviceUnitStateCompactionStrategyTest.java | 90 +++++++++---------- .../channel/ServiceUnitStateDataTest.java | 8 +- .../channel/ServiceUnitStateTest.java | 62 +++++++------ .../ServiceUnitStateCompactionTest.java | 12 +-- 8 files changed, 156 insertions(+), 126 deletions(-) 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 43a3b0a3e325b..9e0c51f0ba7a7 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 @@ -34,29 +34,37 @@ public enum ServiceUnitState { Owned, // owned by a broker (terminal state) - Assigned, // the ownership is assigned(but the assigned broker has not been notified the ownership yet) + Assigning, // the ownership is being assigned (e.g. the new ownership is being notified to the target broker) - Released, // the source broker's ownership has been released (e.g. the topic connections are closed) + Releasing, // the source broker's ownership is being released (e.g. the topic connections are being closed) - Splitting, // the service unit(e.g. bundle) is in the process of splitting. + Splitting, // the service unit is in the process of splitting. (e.g. the metadata store is being updated) Deleted; // deleted in the system (semi-terminal state) private static final Map> validTransitions = 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, Assigned, Released, Splitting, Deleted), - Free, Set.of(Assigned, Init), - Owned, Set.of(Assigned, Splitting, Released), - Assigned, Set.of(Owned, Released), - Released, Set.of(Owned, Free), + Init, Set.of(Free, Owned, Assigning, Releasing, Splitting, Deleted), + Free, Set.of(Assigning, Init), + Owned, Set.of(Assigning, Splitting, Releasing), + Assigning, Set.of(Owned, Releasing), + Releasing, Set.of(Owned, 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); return transitions.contains(to); } + public static boolean isInFlightState(ServiceUnitState state) { + return inFlightStates.contains(state); + } + } 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 f1656bbadef5a..b316019d0cadb 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 @@ -21,12 +21,12 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +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.ServiceUnitStateChannelImpl.ChannelState.Closed; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Constructed; @@ -433,7 +433,7 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { case Owned, Splitting -> { return CompletableFuture.completedFuture(Optional.of(data.broker())); } - case Assigned, Released -> { + case Assigning, Releasing -> { return deferGetOwnerRequest(serviceUnit).thenApply( broker -> broker == null ? Optional.empty() : Optional.of(broker)); } @@ -455,7 +455,7 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str EventType eventType = Assign; eventCounters.get(eventType).getTotal().incrementAndGet(); CompletableFuture getOwnerRequest = deferGetOwnerRequest(serviceUnit); - pubAsync(serviceUnit, new ServiceUnitStateData(Assigned, broker)) + pubAsync(serviceUnit, new ServiceUnitStateData(Assigning, broker)) .whenComplete((__, ex) -> { if (ex != null) { getOwnerRequests.remove(serviceUnit, getOwnerRequest); @@ -475,10 +475,10 @@ public CompletableFuture publishUnloadEventAsync(Unload unload) { CompletableFuture future; if (isTransferCommand(unload)) { future = pubAsync(serviceUnit, new ServiceUnitStateData( - Assigned, unload.destBroker().get(), unload.sourceBroker())); + Assigning, unload.destBroker().get(), unload.sourceBroker())); } else { future = pubAsync(serviceUnit, new ServiceUnitStateData( - Released, unload.sourceBroker())); + Releasing, unload.sourceBroker())); } return future.whenComplete((__, ex) -> { @@ -511,8 +511,8 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { try { switch (state) { case Owned -> handleOwnEvent(serviceUnit, data); - case Assigned -> handleAssignEvent(serviceUnit, data); - case Released -> handleReleaseEvent(serviceUnit, data); + case Assigning -> handleAssignEvent(serviceUnit, data); + case Releasing -> handleReleaseEvent(serviceUnit, data); case Splitting -> handleSplitEvent(serviceUnit, data); case Deleted -> handleDeleteEvent(serviceUnit, data); case Free -> handleFreeEvent(serviceUnit, data); @@ -599,7 +599,7 @@ private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) { private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) { if (isTargetBroker(data.broker())) { ServiceUnitStateData next = new ServiceUnitStateData( - isTransferCommand(data) ? Released : Owned, data.broker(), data.sourceBroker()); + isTransferCommand(data) ? Releasing : Owned, data.broker(), data.sourceBroker()); pubAsync(serviceUnit, next) .whenComplete((__, e) -> log(e, serviceUnit, data, next)); } @@ -964,10 +964,18 @@ private void doCleanup(String broker) throws ExecutionException, InterruptedExce for (var etr : tableview.entrySet()) { var stateData = etr.getValue(); var serviceUnit = etr.getKey(); - if (StringUtils.equals(broker, stateData.broker()) - || StringUtils.equals(broker, stateData.sourceBroker())) { - overrideOwnership(serviceUnit, stateData, availableBrokers); - orphanServiceUnitCleanupCnt++; + var state = state(stateData); + if (StringUtils.equals(broker, stateData.broker())) { + if (ServiceUnitState.isInFlightState(state) || state == Owned) { + overrideOwnership(serviceUnit, stateData, availableBrokers); + orphanServiceUnitCleanupCnt++; + } + + } else if (StringUtils.equals(broker, stateData.sourceBroker())) { + if (ServiceUnitState.isInFlightState(state)) { + overrideOwnership(serviceUnit, stateData, availableBrokers); + orphanServiceUnitCleanupCnt++; + } } } @@ -1002,14 +1010,14 @@ private Optional getOverrideStateData(String serviceUnit, if (isTransferCommand(orphanData)) { // rollback to the src return Optional.of(new ServiceUnitStateData(Owned, orphanData.sourceBroker(), true)); - } else if (orphanData.state() == Assigned) { // assign + } else if (orphanData.state() == Assigning) { // assign // roll-forward to another broker Optional selectedBroker = brokerSelector.select(availableBrokers, null, context); if (selectedBroker.isEmpty()) { return Optional.empty(); } return Optional.of(new ServiceUnitStateData(Owned, selectedBroker.get(), true)); - } else if (orphanData.state() == Splitting || orphanData.state() == Released) { + } else if (orphanData.state() == Splitting || orphanData.state() == Releasing) { // rollback to the target broker for split and unload return Optional.of(new ServiceUnitStateData(Owned, orphanData.broker(), true)); } else { 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/ServiceUnitStateCompactionStrategy.java index 1c3196df97c66..d2a585af9d9d5 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/ServiceUnitStateCompactionStrategy.java @@ -66,20 +66,20 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to switch (prevState) { case Owned: switch (state) { - case Assigned: + case Assigning: return invalidTransfer(from, to); case Splitting: - case Released: + case Releasing: return isNotBlank(to.sourceBroker()) || targetNotEquals(from, to); } - case Assigned: + case Assigning: switch (state) { - case Released: + case Releasing: return isBlank(to.sourceBroker()) || notEquals(from, to); case Owned: return isNotBlank(to.sourceBroker()) || targetNotEquals(from, to); } - case Released: + case Releasing: switch (state) { case Owned: case Free: @@ -92,7 +92,7 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to } case Free: switch (state) { - case Assigned: + case Assigning: return isNotBlank(to.sourceBroker()); } } 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 255488f0bbc78..49eee6ecb7aef 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 @@ -18,12 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.channel; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +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.ServiceUnitStateChannelImpl.EventType.Assign; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Split; @@ -717,6 +717,10 @@ public void handleBrokerDeletionEventTest() waitUntilNewOwner(channel1, bundle2, broker); waitUntilNewOwner(channel2, bundle2, broker); + channel1.publishUnloadEventAsync(new Unload(broker, bundle1, Optional.of(lookupServiceAddress2))); + waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); + waitUntilNewOwner(channel2, bundle1, lookupServiceAddress2); + // test stable metadata state leaderChannel.handleMetadataSessionEvent(SessionReestablished); followerChannel.handleMetadataSessionEvent(SessionReestablished); @@ -741,7 +745,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 1, 0, - 2, + 1, 0, 1, 0, @@ -768,7 +772,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 1, 0, - 2, + 1, 0, 2, 0, @@ -785,7 +789,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 1, 0, - 2, + 1, 0, 2, 0, @@ -804,7 +808,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 1, 0, - 2, + 1, 0, 3, 0, @@ -823,7 +827,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 4, + 3, 0, 3, 0, @@ -849,7 +853,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 4, + 3, 0, 3, 1, @@ -920,7 +924,7 @@ public void conflictAndCompactionTest() throws ExecutionException, InterruptedEx @Test(priority = 11) public void ownerLookupCountTests() throws IllegalAccessException { - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigned, "b1")); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, "b1")); channel1.getOwnerAsync(bundle); channel1.getOwnerAsync(bundle); @@ -929,7 +933,7 @@ public void ownerLookupCountTests() throws IllegalAccessException { channel1.getOwnerAsync(bundle); channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Released, "b1")); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, "b1")); channel1.getOwnerAsync(bundle); channel1.getOwnerAsync(bundle); @@ -1386,12 +1390,12 @@ private static void validateHandlerCounters(ServiceUnitStateChannel channel, .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> { // wait until true - assertEquals(assignedT, handlerCounters.get(Assigned).getTotal().get()); - assertEquals(assignedF, handlerCounters.get(Assigned).getFailure().get()); + assertEquals(assignedT, handlerCounters.get(Assigning).getTotal().get()); + assertEquals(assignedF, handlerCounters.get(Assigning).getFailure().get()); assertEquals(ownedT, handlerCounters.get(Owned).getTotal().get()); assertEquals(ownedF, handlerCounters.get(Owned).getFailure().get()); - assertEquals(releasedT, handlerCounters.get(Released).getTotal().get()); - assertEquals(releasedF, handlerCounters.get(Released).getFailure().get()); + assertEquals(releasedT, handlerCounters.get(Releasing).getTotal().get()); + assertEquals(releasedF, handlerCounters.get(Releasing).getFailure().get()); assertEquals(splittingT, handlerCounters.get(Splitting).getTotal().get()); assertEquals(splittingF, handlerCounters.get(Splitting).getFailure().get()); assertEquals(freeT, handlerCounters.get(Free).getTotal().get()); @@ -1443,9 +1447,9 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> { // wait until true - assertEquals(assigned, ownerLookUpCounters.get(Assigned).get()); + assertEquals(assigned, ownerLookUpCounters.get(Assigning).get()); assertEquals(owned, ownerLookUpCounters.get(Owned).get()); - assertEquals(released, ownerLookUpCounters.get(Released).get()); + assertEquals(released, ownerLookUpCounters.get(Releasing).get()); assertEquals(splitting, ownerLookUpCounters.get(Splitting).get()); assertEquals(free, ownerLookUpCounters.get(Free).get()); assertEquals(deleted, ownerLookUpCounters.get(Deleted).get()); 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/ServiceUnitStateCompactionStrategyTest.java index 4a5ac391a917c..1a4aba15f9e6f 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/ServiceUnitStateCompactionStrategyTest.java @@ -18,12 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.channel; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +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.junit.jupiter.api.Assertions.assertFalse; import static org.testng.Assert.assertTrue; @@ -58,62 +58,62 @@ public void test() throws InterruptedException { assertTrue(strategy.shouldKeepLeft(data(Init), data(Init))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Free))); - assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigned))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Assigning))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Owned))); - assertFalse(strategy.shouldKeepLeft(data(Init), data(Released))); + assertFalse(strategy.shouldKeepLeft(data(Init), data(Releasing))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Splitting))); assertFalse(strategy.shouldKeepLeft(data(Init), data(Deleted))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Init))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, "dst1"), data(Owned, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, src, dst))); - assertFalse(strategy.shouldKeepLeft(data(Assigned, dst), data(Owned, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, src, dst), data(Released, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, src, "dst1"), data(Released, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Assigned, "src1", dst), data(Released, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Assigned, src, dst), data(Released, src, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Splitting, dst))); - assertTrue(strategy.shouldKeepLeft(data(Assigned), data(Deleted, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigning), data(Init))); + assertTrue(strategy.shouldKeepLeft(data(Assigning), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Assigning), data(Assigning))); + assertTrue(strategy.shouldKeepLeft(data(Assigning, "dst1"), data(Owned, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Assigning, dst), data(Owned, src, dst))); + assertFalse(strategy.shouldKeepLeft(data(Assigning, dst), data(Owned, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigning, src, dst), data(Releasing, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigning, src, "dst1"), data(Releasing, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Assigning, "src1", dst), data(Releasing, "src2", dst))); + assertFalse(strategy.shouldKeepLeft(data(Assigning, src, dst), data(Releasing, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigning), data(Splitting, dst))); + assertTrue(strategy.shouldKeepLeft(data(Assigning), data(Deleted, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigned, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, src, dst))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigned, dst, "dst1"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Assigning, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigning, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigning, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigning, dst, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Assigning, dst, "dst1"))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Owned), data(Released, dst))); - assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Released, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Released, "dst2"))); - assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Released, dst))); - assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Released, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned), data(Releasing, dst))); + assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Releasing, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Releasing, "dst2"))); + assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Releasing, dst))); + assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Releasing, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned, src, "dst1"), data(Splitting, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Owned, "dst1"), data(Splitting, "dst2"))); assertFalse(strategy.shouldKeepLeft(data(Owned, dst), data(Splitting, dst))); assertFalse(strategy.shouldKeepLeft(data(Owned, src, dst), data(Splitting, dst))); assertTrue(strategy.shouldKeepLeft(data(Owned), data(Deleted, dst))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Init))); - assertFalse(strategy.shouldKeepLeft(data(Released), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Free, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Released, "src1", dst), data(Free, "src2", dst))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Released, "dst1"), data(Owned, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Released, src, "dst1"), data(Owned, src, "dst2"))); - assertTrue(strategy.shouldKeepLeft(data(Released, "src1", dst), data(Owned, "src2", dst))); - assertFalse(strategy.shouldKeepLeft(data(Released, src, dst), data(Owned, src, dst))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Released))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Splitting))); - assertTrue(strategy.shouldKeepLeft(data(Released), data(Deleted, dst))); + assertTrue(strategy.shouldKeepLeft(data(Releasing), data(Init))); + assertFalse(strategy.shouldKeepLeft(data(Releasing), data(Free))); + assertTrue(strategy.shouldKeepLeft(data(Releasing, "dst1"), data(Free, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Releasing, "src1", dst), data(Free, "src2", dst))); + assertTrue(strategy.shouldKeepLeft(data(Releasing), data(Assigning))); + assertTrue(strategy.shouldKeepLeft(data(Releasing, "dst1"), data(Owned, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Releasing, src, "dst1"), data(Owned, src, "dst2"))); + assertTrue(strategy.shouldKeepLeft(data(Releasing, "src1", dst), data(Owned, "src2", dst))); + assertFalse(strategy.shouldKeepLeft(data(Releasing, src, dst), data(Owned, src, dst))); + assertTrue(strategy.shouldKeepLeft(data(Releasing), data(Releasing))); + assertTrue(strategy.shouldKeepLeft(data(Releasing), data(Splitting))); + assertTrue(strategy.shouldKeepLeft(data(Releasing), data(Deleted, dst))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigned))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Assigning))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Releasing))); assertTrue(strategy.shouldKeepLeft(data(Splitting), data(Splitting))); assertTrue(strategy.shouldKeepLeft(data(Splitting, src, "dst1"), data(Deleted, src, "dst2"))); assertTrue(strategy.shouldKeepLeft(data(Splitting, "dst1"), data(Deleted, "dst2"))); @@ -123,18 +123,18 @@ public void test() throws InterruptedException { assertFalse(strategy.shouldKeepLeft(data(Deleted), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Free))); - assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Assigned))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Assigning))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Releasing))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Splitting))); assertTrue(strategy.shouldKeepLeft(data(Deleted), data(Deleted))); assertFalse(strategy.shouldKeepLeft(data(Free), data(Init))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Free))); - assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigned, src, dst))); + assertFalse(strategy.shouldKeepLeft(data(Free), data(Assigning))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Assigning, src, dst))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Owned))); - assertTrue(strategy.shouldKeepLeft(data(Free), data(Released))); + assertTrue(strategy.shouldKeepLeft(data(Free), data(Releasing))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Splitting))); assertTrue(strategy.shouldKeepLeft(data(Free), data(Deleted))); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java index b39f7c2b8d8c6..9617c8a8c2bd0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.channel; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; @@ -41,8 +41,8 @@ public void testConstructors() throws InterruptedException { Thread.sleep(10); - ServiceUnitStateData data2 = new ServiceUnitStateData(Assigned, "A", "B"); - assertEquals(data2.state(), Assigned); + ServiceUnitStateData data2 = new ServiceUnitStateData(Assigning, "A", "B"); + assertEquals(data2.state(), Assigning); assertEquals(data2.broker(), "A"); assertEquals(data2.sourceBroker(), "B"); assertThat(data2.timestamp()).isGreaterThan(data1.timestamp()); @@ -66,7 +66,7 @@ public void testEmptyBroker() { @Test public void jsonWriteAndReadTest() throws JsonProcessingException { ObjectMapper mapper = ObjectMapperFactory.create(); - final ServiceUnitStateData src = new ServiceUnitStateData(Assigned, "A", "B"); + final ServiceUnitStateData src = new ServiceUnitStateData(Assigning, "A", "B"); String json = mapper.writeValueAsString(src); ServiceUnitStateData dst = mapper.readValue(json, ServiceUnitStateData.class); assertEquals(dst, src); 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 1a913cdbaabee..f5f1fe7bc575f 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 @@ -18,12 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.channel; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +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.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -31,6 +31,16 @@ @Test(groups = "broker") public class ServiceUnitStateTest { + @Test + public void testInFlights() { + assertFalse(ServiceUnitState.isInFlightState(Init)); + assertFalse(ServiceUnitState.isInFlightState(Free)); + assertFalse(ServiceUnitState.isInFlightState(Owned)); + assertTrue(ServiceUnitState.isInFlightState(Assigning)); + assertTrue(ServiceUnitState.isInFlightState(Releasing)); + assertTrue(ServiceUnitState.isInFlightState(Splitting)); + assertFalse(ServiceUnitState.isInFlightState(Deleted)); + } @Test public void testTransitions() { @@ -38,56 +48,56 @@ public void testTransitions() { assertFalse(ServiceUnitState.isValidTransition(Init, Init)); assertTrue(ServiceUnitState.isValidTransition(Init, Free)); assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Init, Assigned)); - assertTrue(ServiceUnitState.isValidTransition(Init, Released)); + 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, Assigned)); - assertFalse(ServiceUnitState.isValidTransition(Free, Released)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigning)); + assertFalse(ServiceUnitState.isValidTransition(Free, Releasing)); assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Free, Deleted)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Init)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Free)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Assigned)); - assertTrue(ServiceUnitState.isValidTransition(Assigned, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Assigned, Released)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Assigned, Deleted)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Init)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Free)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Releasing)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted)); assertFalse(ServiceUnitState.isValidTransition(Owned, Init)); assertFalse(ServiceUnitState.isValidTransition(Owned, Free)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Assigned)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Assigning)); assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Released)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing)); assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); - assertFalse(ServiceUnitState.isValidTransition(Released, Init)); - assertTrue(ServiceUnitState.isValidTransition(Released, Free)); - assertFalse(ServiceUnitState.isValidTransition(Released, Assigned)); - assertTrue(ServiceUnitState.isValidTransition(Released, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Released, Released)); - assertFalse(ServiceUnitState.isValidTransition(Released, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Released, Deleted)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Init)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Free)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Assigning)); + assertTrue(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, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning)); assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Released)); + 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, Assigned)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Released)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting)); assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted)); } 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 61299bc79dabf..880a237d412d7 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 @@ -21,8 +21,8 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Deleted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Init; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +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.isValidTransition; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; @@ -531,7 +531,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { AtomicBoolean handledReleased = new AtomicBoolean(false); slowTV.listen((k, v) -> { - if (v.state() == Assigned) { + if (v.state() == Assigning) { try { // Stuck at handling Assigned handledReleased.set(false); @@ -540,7 +540,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { } catch (InterruptedException e) { throw new RuntimeException(e); } - } else if (v.state() == Released) { + } else if (v.state() == Releasing) { handledReleased.set(true); } }); @@ -563,10 +563,10 @@ public void testSlowTableviewAfterCompaction() throws Exception { String dst = "broker1"; producer.newMessage().key(bundle).value(new ServiceUnitStateData(Owned, src)).send(); for (int i = 0; i < 3; i++) { - var assignedStateData = new ServiceUnitStateData(Assigned, dst, src); + var assignedStateData = new ServiceUnitStateData(Assigning, dst, src); producer.newMessage().key(bundle).value(assignedStateData).send(); producer.newMessage().key(bundle).value(assignedStateData).send(); - var releasedStateData = new ServiceUnitStateData(Released, dst, src); + var releasedStateData = new ServiceUnitStateData(Releasing, dst, src); producer.newMessage().key(bundle).value(releasedStateData).send(); producer.newMessage().key(bundle).value(releasedStateData).send(); var ownedStateData = new ServiceUnitStateData(Owned, dst, src); From 06897714df9825492425cbc21c7312c22e24c22a Mon Sep 17 00:00:00 2001 From: Heesung Sohn Date: Wed, 22 Feb 2023 10:00:22 -0800 Subject: [PATCH 7/7] resolved minor comments --- .../extensions/channel/ServiceUnitState.java | 2 +- .../channel/ServiceUnitStateChannelImpl.java | 2 +- .../ExtensibleLoadManagerImplTest.java | 18 +++++++++--------- .../ServiceUnitStateCompactionTest.java | 1 - 4 files changed, 11 insertions(+), 12 deletions(-) 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 9e0c51f0ba7a7..92fef8f65992a 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 @@ -24,7 +24,7 @@ /** * Defines the possible states for service units. * - * Refer to Service Unit State Channel in https://github.com/apache/pulsar/issues/16691 for additional details. + * @see Service Unit State Channel for additional details. */ public enum ServiceUnitState { 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 b316019d0cadb..9f205f85c5454 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 @@ -959,7 +959,7 @@ private void doCleanup(String broker) throws ExecutionException, InterruptedExce log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); - var availableBrokers = new HashSet(brokerRegistry.getAvailableBrokersAsync() + var availableBrokers = new HashSet<>(brokerRegistry.getAvailableBrokersAsync() .get(inFlightStateWaitingTimeInMillis, MILLISECONDS)); for (var etr : tableview.entrySet()) { var stateData = etr.getValue(); 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 39756006b24bb..001aac34a4ba2 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 @@ -433,6 +433,7 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, feature=max_ema, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=4.0}] dimensions=[{broker=localhost, feature=max, metric=loadBalancing}], metrics=[{brk_lb_resource_usage=0.04}] dimensions=[{broker=localhost, metric=bundleUnloading}], metrics=[{brk_lb_unload_broker_total=2, brk_lb_unload_bundle_total=3}] + dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Balanced, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=NoBundles, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=4}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=CoolDown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=5}] @@ -442,7 +443,6 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Skip}], metrics=[{brk_lb_unload_broker_breakdown_total=9}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Overloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=bundleUnloading, reason=Underloaded, result=Success}], metrics=[{brk_lb_unload_broker_breakdown_total=2}] - dimensions=[{broker=localhost, metric=bundleUnloading, reason=Unknown, result=Failure}], metrics=[{brk_lb_unload_broker_breakdown_total=10}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=avg}], metrics=[{brk_lb_resource_usage_stats=1.5}] dimensions=[{broker=localhost, feature=max_ema, metric=bundleUnloading, stat=std}], metrics=[{brk_lb_resource_usage_stats=0.3}] dimensions=[{broker=localhost, metric=bundlesSplit}], metrics=[{brk_lb_bundles_split_total=35}] @@ -453,14 +453,14 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, metric=bundlesSplit, reason=Admin, result=Success}], metrics=[{brk_lb_bundles_split_breakdown_total=5}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Balanced, result=Skip}], metrics=[{brk_lb_bundles_split_breakdown_total=6}] dimensions=[{broker=localhost, metric=bundlesSplit, reason=Unknown, result=Failure}], metrics=[{brk_lb_bundles_split_breakdown_total=7}] - dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] - dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] dimensions=[{broker=localhost, metric=assign, result=Empty}], metrics=[{brk_lb_assign_broker_breakdown_total=2}] + dimensions=[{broker=localhost, metric=assign, result=Skip}], metrics=[{brk_lb_assign_broker_breakdown_total=3}] + dimensions=[{broker=localhost, metric=assign, result=Success}], metrics=[{brk_lb_assign_broker_breakdown_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Init}], metrics=[{brk_sunit_state_chn_owner_lookup_total=1}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Free}], metrics=[{brk_sunit_state_chn_owner_lookup_total=2}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Owned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=3}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigned}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] - dimensions=[{broker=localhost, metric=sunitStateChn, state=Released}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Assigning}], metrics=[{brk_sunit_state_chn_owner_lookup_total=4}] + dimensions=[{broker=localhost, metric=sunitStateChn, state=Releasing}], metrics=[{brk_sunit_state_chn_owner_lookup_total=5}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Splitting}], metrics=[{brk_sunit_state_chn_owner_lookup_total=6}] dimensions=[{broker=localhost, metric=sunitStateChn, state=Deleted}], metrics=[{brk_sunit_state_chn_owner_lookup_total=7}] dimensions=[{broker=localhost, event=Assign, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_event_publish_ops_total=1}] @@ -475,10 +475,10 @@ SplitDecision.Reason.Balanced, new MutableLong(6) dimensions=[{broker=localhost, event=Free, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=4}] dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=5}] dimensions=[{broker=localhost, event=Owned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=6}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] - dimensions=[{broker=localhost, event=Assigned, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] - dimensions=[{broker=localhost, event=Released, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] + dimensions=[{broker=localhost, event=Assigning, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=7}] + dimensions=[{broker=localhost, event=Assigning, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=8}] + dimensions=[{broker=localhost, event=Releasing, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=9}] + dimensions=[{broker=localhost, event=Releasing, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=10}] dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=11}] dimensions=[{broker=localhost, event=Splitting, metric=sunitStateChn, result=Failure}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=12}] dimensions=[{broker=localhost, event=Deleted, metric=sunitStateChn, result=Total}], metrics=[{brk_sunit_state_chn_subscribe_ops_total=13}] 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 880a237d412d7..4c1d4f7d2a89d 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 @@ -536,7 +536,6 @@ public void testSlowTableviewAfterCompaction() throws Exception { // Stuck at handling Assigned handledReleased.set(false); semaphore.acquire(); - //Thread.sleep(5000); } catch (InterruptedException e) { throw new RuntimeException(e); }