From 9392e6e1d935363e45e9e1a8db257d89c3e495f0 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Thu, 15 May 2025 10:05:11 -0700 Subject: [PATCH 01/11] [Message Prioritization] - Add currentReplicaNumber metadata for message prioritization --- .../stages/MessageGenerationPhase.java | 138 +++++- .../java/org/apache/helix/model/Message.java | 19 +- .../org/apache/helix/util/MessageUtil.java | 28 ++ .../TestPrioritizationMessageGeneration.java | 422 ++++++++++++++++++ 4 files changed, 603 insertions(+), 4 deletions(-) create mode 100644 helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 859c6679e9..b19752e9c6 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -163,6 +163,16 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr // desired-state->list of generated-messages Map> messageMap = new HashMap<>(); + // Calculate metadata for message prioritization + int totalInstanceCount = instanceStateMap.keySet().size(); + + // Get pending upward state transition messages to second top or top state + List pendingUpwardStateTransitionMessages = getPendingUpwardStateTransitionMessages( + resourceName, partition, currentStateOutput, stateModelDef); + + // Initialize replica counter for prioritization + int initialReplicaNumber = totalInstanceCount - pendingUpwardStateTransitionMessages.size(); + for (String instanceName : instanceStateMap.keySet()) { Set staleMessages = cache.getStaleMessagesByInstance(instanceName); @@ -250,17 +260,32 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr pendingMessage, manager, resource, partition, sessionIdMap, instanceName, stateModelDef, cancellationMessage, isCancellationEnabled); } else { + // Default currentReplicaNumber is -1 (no prioritization) + int currentReplicaNumber = -1; + // Check if this is an upward state transition from non-second top state to second top + // or top state + if (isUpwardStateTransition(currentState, nextState, stateModelDef) + && !stateModelDef.getSecondTopStates().contains(currentState) + && (isSecondTopState(nextState, stateModelDef) + || isTopState(nextState, stateModelDef)) + && !isInPendingMessages(resourceName, partition, instanceName, currentState, + nextState, pendingUpwardStateTransitionMessages)) { + + // Assign the replica number for prioritization + currentReplicaNumber = initialReplicaNumber--; + } + // Create new state transition message message = MessageUtil .createStateTransitionMessage(manager.getInstanceName(), manager.getSessionId(), resource, partition.getPartitionName(), instanceName, currentState, nextState, - sessionIdMap.get(instanceName), stateModelDef.getId()); + sessionIdMap.get(instanceName), stateModelDef.getId(), currentReplicaNumber); if (logger.isDebugEnabled()) { LogUtil.logDebug(logger, _eventId, String.format( - "Resource %s partition %s for instance %s with currentState %s and nextState %s", + "Resource %s partition %s for instance %s with currentState %s, nextState %s and currentReplicaNumber %d", resource.getResourceName(), partition.getPartitionName(), instanceName, - currentState, nextState)); + currentState, nextState, currentReplicaNumber)); } } } @@ -290,6 +315,113 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr } // end of for-each-partition } + /** + * Check if a state is a top state + * @param state The state to check + * @param stateModelDef The state model definition + * @return True if it's a top state, false otherwise + */ + private boolean isTopState(String state, StateModelDefinition stateModelDef) { + return stateModelDef.getTopState().contains(state); + } + + /** + * Check if a state is a second top state + * @param state The state to check + * @param stateModelDef The state model definition + * @return True if it's a second top state, false otherwise + */ + private boolean isSecondTopState(String state, StateModelDefinition stateModelDef) { + return stateModelDef.getSecondTopStates().contains(state); + } + + /** + * Check if a state transition is already in the pending messages + * @param resourceName The resource name + * @param partition The partition + * @param instanceName The instance name + * @param fromState The from state + * @param toState The to state + * @param pendingMessages The list of pending messages + * @return True if the state transition is already in pending messages, false otherwise + */ + private boolean isInPendingMessages(String resourceName, Partition partition, String instanceName, + String fromState, String toState, List pendingMessages) { + + for (Message message : pendingMessages) { + if (message.getResourceName().equals(resourceName) + && message.getPartitionName().equals(partition.getPartitionName()) + && message.getTgtName().equals(instanceName) && message.getFromState().equals(fromState) + && message.getToState().equals(toState)) { + return true; + } + } + + return false; + } + + /** + * Check if a state transition is upward + * @param fromState The from state + * @param toState The to state + * @param stateModelDef The state model definition + * @return True if it's an upward state transition, false otherwise + */ + private boolean isUpwardStateTransition(String fromState, String toState, + StateModelDefinition stateModelDef) { + + if (fromState == null || toState == null) { + return false; + } + + Map statePriorityMap = stateModelDef.getStatePriorityMap(); + + Integer fromStateWeight = statePriorityMap.get(fromState); + Integer toStateWeight = statePriorityMap.get(toState); + + if (fromStateWeight == null || toStateWeight == null) { + return false; + } + + return toStateWeight < fromStateWeight; + } + + /** + * Get pending upward state transition messages from non-second top state to second top or top + * state + * @param resourceName The resource name + * @param partition The partition + * @param currentStateOutput The current state output + * @param stateModelDef The state model definition + * @return List of pending messages for upward state transitions + */ + private List getPendingUpwardStateTransitionMessages(String resourceName, + Partition partition, CurrentStateOutput currentStateOutput, + StateModelDefinition stateModelDef) { + List pendingUpwardSTMessages = new ArrayList<>(); + + // Instance -> PendingMessage + Map pendingMessages = + currentStateOutput.getPendingMessageMap(resourceName, partition); + + if (pendingMessages != null && !pendingMessages.isEmpty()) { + for (Message message : pendingMessages.values()) { + String fromState = message.getFromState(); + String toState = message.getToState(); + + // Check if it's an upward state transition from non-second top state to second top or top + // state + if (isUpwardStateTransition(fromState, toState, stateModelDef) + && !isSecondTopState(fromState, stateModelDef) + && (isSecondTopState(toState, stateModelDef) || isTopState(toState, stateModelDef))) { + pendingUpwardSTMessages.add(message); + } + } + } + + return pendingUpwardSTMessages; + } + private boolean shouldCreateSTCancellation(Message pendingMessage, String desiredState, String initialState) { if (pendingMessage == null) { diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java index acf0758217..8a15452ea5 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Message.java +++ b/helix-core/src/main/java/org/apache/helix/model/Message.java @@ -104,7 +104,8 @@ public enum Attributes { RELAY_FROM, EXPIRY_PERIOD, SRC_CLUSTER, - ST_REBALANCE_TYPE + ST_REBALANCE_TYPE, + CURRENT_REPLICA_NUMBER } /** @@ -935,6 +936,22 @@ public void setSrcClusterName(String clusterName) { _record.setSimpleField(Attributes.SRC_CLUSTER.name(), clusterName); } + /** + * Set the currentReplicaNumber for transition-related messages + * @param currentReplicaNumber the replica count + */ + public void setCurrentReplicaNumber(int currentReplicaNumber) { + _record.setIntField(Attributes.CURRENT_REPLICA_NUMBER.name(), currentReplicaNumber); + } + + /** + * Get the "CurrentReplicaNumber" for transition-related messages + * @return CurrentReplicaNumber, or null for other message types + */ + public int getCurrentReplicaNumber() { + return _record.getIntField(Attributes.CURRENT_REPLICA_NUMBER.name(), -1); + } + /** * Check if this message is targetted for a controller * @return true if this is a controller message, false otherwise diff --git a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java index 94de8331b1..a82c1d58ba 100644 --- a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java +++ b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java @@ -136,4 +136,32 @@ private static Message createStateTransitionMessage(Message.MessageType messageT return message; } + + /** + * Create a state transition message with currentReplicaNumber for prioritization + * @param msgSender message sender + * @param sessionId session id + * @param resource resource + * @param partitionName partition name + * @param instanceName instance name + * @param fromState from state + * @param toState to state + * @param sessionIdForInstance session id for instance + * @param stateModelDefName state model def name + * @param currentReplicaNumber the current replica number (for prioritization) + * @return message + */ + public static Message createStateTransitionMessage(String msgSender, String sessionId, + Resource resource, String partitionName, String instanceName, String fromState, + String toState, String sessionIdForInstance, String stateModelDefName, + int currentReplicaNumber) { + + Message message = createStateTransitionMessage(msgSender, sessionId, resource, partitionName, + instanceName, fromState, toState, sessionIdForInstance, stateModelDefName); + + // Set the current replica number for prioritization + message.setCurrentReplicaNumber(currentReplicaNumber); + + return message; + } } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java new file mode 100644 index 0000000000..9c16c0674c --- /dev/null +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -0,0 +1,422 @@ +package org.apache.helix.controller.stages; + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixManager; +import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; +import org.apache.helix.model.ClusterConfig; +import org.apache.helix.model.CurrentState; +import org.apache.helix.model.LiveInstance; +import org.apache.helix.model.Message; +import org.apache.helix.model.Partition; +import org.apache.helix.model.Resource; +import org.apache.helix.model.StateModelDefinition; +import org.apache.helix.tools.StateModelConfigGenerator; +import org.apache.helix.zookeeper.datamodel.ZNRecord; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestPrioritizationMessageGeneration extends MessageGenerationPhase { + + private static final String TEST_CLUSTER = "TestCluster"; + private static final String TEST_RESOURCE = "TestDB"; + private static final String PARTITION_0 = "TestDB_0"; + private static final String INSTANCE_0 = "localhost_0"; + private static final String INSTANCE_1 = "localhost_1"; + private static final String INSTANCE_2 = "localhost_2"; + private static final String INSTANCE_3 = "localhost_3"; + private static final String SESSION_ID = "123"; + + @Test + public void testPrioritizationForUpwardStateTransition() throws Exception { + + // Test prioritization for upward state transitions from non-second top states + // to second top or top states + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Set up current states + Map currentStateMap = setupCurrentStatesForPrioritizationCase1(); + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants to move some nodes to SLAVE + Map partitionMap = new HashMap<>(); + // No ST messages should be generated as current state is same as desired state + partitionMap.put(INSTANCE_0, "MASTER"); // Master -> Master + partitionMap.put(INSTANCE_1, "SLAVE"); // Slave -> Slave + + // upward to second top + partitionMap.put(INSTANCE_2, "SLAVE"); // Offline -> Slave + partitionMap.put(INSTANCE_3, "SLAVE"); // Offline -> Slave + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 2 messages + Assert.assertEquals(messages.size(), 2); + + // Messages should have replica numbers assigned based on priority + Map replicaNumbers = new HashMap<>(); + for (Message msg : messages) { + int replicaNumber = msg.getCurrentReplicaNumber(); + replicaNumbers.put(msg.getTgtName(), replicaNumber); + } + + // Verify replica numbers are assigned (not -1) for upward transitions + for (String instance : new String[] { + INSTANCE_2, INSTANCE_3 + }) { + Assert.assertTrue(replicaNumbers.get(instance) >= 0, + "Replica number should be assigned for " + instance); + } + + // Verify the replica numbers are decreasing (higher number = higher priority) + Assert.assertTrue(replicaNumbers.get(INSTANCE_3) > replicaNumbers.get(INSTANCE_2), + "Earlier transitions should have higher replica numbers"); + } + + @Test + public void testPrioritizationForUpwardStateTransitionWithAllOfflineInstances() throws Exception { + + // Test prioritization for upward state transitions from non-second top states + // to second top or top states + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Set up current states - multiple instances in OFFLINE state + // We want to test that they get prioritized when transitioning to SLAVE (second top) or MASTER + // (top) + Map currentStateMap = setupCurrentStatesForPrioritizationCase2(); + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants to move some nodes to SLAVE/MASTER + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "SLAVE"); // Offline -> Slave (upward to second top) + partitionMap.put(INSTANCE_1, "MASTER"); // Offline -> Master (upward to top) + partitionMap.put(INSTANCE_2, "SLAVE"); // Offline -> Slave (upward to second top) + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 3 messages + Assert.assertEquals(messages.size(), 3); + + // Messages should have replica numbers assigned based on priority + Map replicaNumbers = new HashMap<>(); + for (Message msg : messages) { + int replicaNumber = msg.getCurrentReplicaNumber(); + replicaNumbers.put(msg.getTgtName(), replicaNumber); + } + + // Verify replica numbers are assigned (not -1) for upward transitions + for (String instance : new String[] { + INSTANCE_0, INSTANCE_1, INSTANCE_2 + }) { + Assert.assertTrue(replicaNumbers.get(instance) >= 0, + "Replica number should be assigned for " + instance); + } + + // Verify the replica numbers are decreasing (higher number = higher priority) + Assert.assertTrue(replicaNumbers.get(INSTANCE_2) > replicaNumbers.get(INSTANCE_0), + "Earlier transitions should have higher replica numbers"); + } + + @Test + public void testNoPrioritizationForNonUpwardTransitions() throws Exception { + // Test that non-upward transitions don't get prioritized + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Set up current states - instance already in SLAVE (second top) + Map currentStateMap = new HashMap<>(); + CurrentState currentState = new CurrentState(TEST_RESOURCE); + currentState.setState(PARTITION_0, "SLAVE"); + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, currentState); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants to move to OFFLINE (downward transition) + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "OFFLINE"); + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + Assert.assertEquals(messages.size(), 1); + + // Should have default replica number (-1) for non-upward transition + Message msg = messages.get(0); + Assert.assertEquals(msg.getCurrentReplicaNumber(), -1, + "Non-upward transitions should not be prioritized"); + } + + @Test + public void testPrioritizationWithPendingMessages() throws Exception { + // Test prioritization when there are already pending messages + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + Map currentStateMap = new HashMap<>(); + + // Instance 0: OFFLINE with pending OFFLINE->SLAVE message + CurrentState currentState0 = new CurrentState(TEST_RESOURCE); + currentState0.setState(PARTITION_0, "OFFLINE"); + currentState0.setSessionId(SESSION_ID); + currentState0.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, currentState0); + + // Create pending message separately + Message pendingMsg = createMessage("OFFLINE", "SLAVE", INSTANCE_0); + pendingMsg.setMsgId(UUID.randomUUID().toString()); + + // Instance 1: OFFLINE with no pending message + CurrentState currentState1 = new CurrentState(TEST_RESOURCE); + currentState1.setState(PARTITION_0, "OFFLINE"); + currentState1.setSessionId(SESSION_ID); + currentState1.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_1, currentState1); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + + // Add pending message to the CurrentStateOutput, not CurrentState + currentStateOutput.setPendingMessage(TEST_RESOURCE, new Partition(PARTITION_0), INSTANCE_0, + pendingMsg); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants both to be SLAVE + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "SLAVE"); + partitionMap.put(INSTANCE_1, "SLAVE"); + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should only generate message for instance 1 (instance 0 already has pending message) + Assert.assertEquals(messages.size(), 1); + Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_1); + + // The message should have a replica number since pending messages are considered + Assert.assertTrue(messages.get(0).getCurrentReplicaNumber() >= 0, + "Should assign replica number considering pending messages"); + } + + @Test + public void testTransitionFromSecondTopState() throws Exception { + // Test that transitions from second top states are not prioritized + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Set up current state - instance in SLAVE state (second top) + Map currentStateMap = new HashMap<>(); + CurrentState currentState = new CurrentState(TEST_RESOURCE); + currentState.setState(PARTITION_0, "SLAVE"); + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, currentState); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants to move to MASTER (top state) + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "MASTER"); + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + Assert.assertEquals(messages.size(), 1); + + // Should have default replica number (-1) as it's from second top state + Message msg = messages.get(0); + Assert.assertEquals(msg.getCurrentReplicaNumber(), -1, + "Transitions from second top states should not be prioritized"); + } + + private ClusterEvent prepareClusterEvent(StateModelDefinition stateModelDef, + CurrentStateOutput currentStateOutput) { + ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); + + event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput); + event.addAttribute(AttributeName.CURRENT_STATE_EXCLUDING_UNKNOWN.name(), currentStateOutput); + + // Mock HelixManager + HelixManager manager = mock(HelixManager.class); + when(manager.getInstanceName()).thenReturn("Controller"); + when(manager.getSessionId()).thenReturn(SESSION_ID); + + // Mock HelixDataAccessor + HelixDataAccessor dataAccessor = mock(HelixDataAccessor.class); + when(manager.getHelixDataAccessor()).thenReturn(dataAccessor); + + event.addAttribute(AttributeName.helixmanager.name(), manager); + + // Setup ResourceControllerDataProvider + ResourceControllerDataProvider cache = mock(ResourceControllerDataProvider.class); + when(cache.getClusterConfig()).thenReturn(new ClusterConfig("TestCluster")); + when(cache.getStateModelDef("MasterSlave")).thenReturn(stateModelDef); + + // Mock live instances + Map liveInstances = new HashMap<>(); + liveInstances.put(INSTANCE_0, createLiveInstance(INSTANCE_0)); + liveInstances.put(INSTANCE_1, createLiveInstance(INSTANCE_1)); + liveInstances.put(INSTANCE_2, createLiveInstance(INSTANCE_2)); + liveInstances.put(INSTANCE_3, createLiveInstance(INSTANCE_3)); + when(cache.getLiveInstances()).thenReturn(liveInstances); + + event.addAttribute(AttributeName.ControllerDataProvider.name(), cache); + + // Setup resources + Map resourceMap = new HashMap<>(); + Resource resource = new Resource(TEST_RESOURCE); + resource.setStateModelDefRef("MasterSlave"); + resource.addPartition(PARTITION_0); + resourceMap.put(TEST_RESOURCE, resource); + event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), resourceMap); + return event; + } + + private Map setupCurrentStatesForPrioritizationCase1() { + Map currentStateMap = new HashMap<>(); + + // All instances in OFFLINE state + for (String instance : new String[] { + INSTANCE_0, INSTANCE_1, INSTANCE_2, INSTANCE_3 + }) { + CurrentState currentState = new CurrentState(TEST_RESOURCE); + if (instance.equals(INSTANCE_0)) { + currentState.setState(PARTITION_0, "MASTER"); + } else if (instance.equals(INSTANCE_1)) { + currentState.setState(PARTITION_0, "SLAVE"); + } else { + currentState.setState(PARTITION_0, "OFFLINE"); + } + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(instance, currentState); + } + + return currentStateMap; + } + + private Map setupCurrentStatesForPrioritizationCase2() { + Map currentStateMap = new HashMap<>(); + + // All instances in OFFLINE state + for (String instance : new String[] { + INSTANCE_0, INSTANCE_1, INSTANCE_2 + }) { + CurrentState currentState = new CurrentState(TEST_RESOURCE); + currentState.setState(PARTITION_0, "OFFLINE"); + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(instance, currentState); + } + + return currentStateMap; + } + + private CurrentStateOutput setupCurrentStateOutput(Map currentStateMap) { + CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + + for (Map.Entry entry : currentStateMap.entrySet()) { + String instance = entry.getKey(); + CurrentState currentState = entry.getValue(); + + currentStateOutput.setCurrentState(TEST_RESOURCE, new Partition(PARTITION_0), instance, + currentState.getState(PARTITION_0)); + } + + return currentStateOutput; + } + + private LiveInstance createLiveInstance(String instanceName) { + // Create LiveInstance with proper ZNRecord initialization + ZNRecord znRecord = new ZNRecord(instanceName); + znRecord.setEphemeralOwner(Long.parseLong(SESSION_ID)); + + LiveInstance liveInstance = new LiveInstance(znRecord); + liveInstance.setSessionId(SESSION_ID); + liveInstance.setHelixVersion("1.0.0"); + liveInstance.setLiveInstance(instanceName); + + return liveInstance; + } + + private Message createMessage(String fromState, String toState, String tgtName) { + Message message = + new Message(Message.MessageType.STATE_TRANSITION, UUID.randomUUID().toString()); + message.setFromState(fromState); + message.setToState(toState); + message.setTgtName(tgtName); + message.setTgtSessionId(SESSION_ID); + message.setResourceName(TEST_RESOURCE); + message.setPartitionName(PARTITION_0); + message.setStateModelDef("MasterSlave"); + message.setSrcName("Controller"); + message.setSrcSessionId(SESSION_ID); + return message; + } +} From 9b80190eaededa4be6888d1fe0965e961646aa55 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Tue, 27 May 2025 16:11:10 -0700 Subject: [PATCH 02/11] Update based on review/feedback --- .../stages/MessageGenerationPhase.java | 69 ++------------ .../java/org/apache/helix/model/Message.java | 18 ++-- .../helix/model/StateModelDefinition.java | 24 +++++ .../org/apache/helix/util/MessageUtil.java | 92 ++++++++++--------- 4 files changed, 93 insertions(+), 110 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index b19752e9c6..e984f5bbc6 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -164,14 +164,14 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr Map> messageMap = new HashMap<>(); // Calculate metadata for message prioritization - int totalInstanceCount = instanceStateMap.keySet().size(); + int partitionInstanceCount = instanceStateMap.keySet().size(); // Get pending upward state transition messages to second top or top state - List pendingUpwardStateTransitionMessages = getPendingUpwardStateTransitionMessages( + List pendingUpwardStateTransitionMessagesToTopOrSecondTopStates = getPendingTransitionsToTopOrSecondTopStates( resourceName, partition, currentStateOutput, stateModelDef); // Initialize replica counter for prioritization - int initialReplicaNumber = totalInstanceCount - pendingUpwardStateTransitionMessages.size(); + int initialReplicaNumber = partitionInstanceCount - pendingUpwardStateTransitionMessagesToTopOrSecondTopStates.size(); for (String instanceName : instanceStateMap.keySet()) { @@ -260,16 +260,14 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr pendingMessage, manager, resource, partition, sessionIdMap, instanceName, stateModelDef, cancellationMessage, isCancellationEnabled); } else { - // Default currentReplicaNumber is -1 (no prioritization) + // Default currentReplicaNumber is -1 (provides metadata for participant-side prioritization) int currentReplicaNumber = -1; // Check if this is an upward state transition from non-second top state to second top // or top state - if (isUpwardStateTransition(currentState, nextState, stateModelDef) + if (stateModelDef.isUpwardStateTransition(currentState, nextState) && !stateModelDef.getSecondTopStates().contains(currentState) && (isSecondTopState(nextState, stateModelDef) - || isTopState(nextState, stateModelDef)) - && !isInPendingMessages(resourceName, partition, instanceName, currentState, - nextState, pendingUpwardStateTransitionMessages)) { + || isTopState(nextState, stateModelDef))) { // Assign the replica number for prioritization currentReplicaNumber = initialReplicaNumber--; @@ -335,57 +333,6 @@ private boolean isSecondTopState(String state, StateModelDefinition stateModelDe return stateModelDef.getSecondTopStates().contains(state); } - /** - * Check if a state transition is already in the pending messages - * @param resourceName The resource name - * @param partition The partition - * @param instanceName The instance name - * @param fromState The from state - * @param toState The to state - * @param pendingMessages The list of pending messages - * @return True if the state transition is already in pending messages, false otherwise - */ - private boolean isInPendingMessages(String resourceName, Partition partition, String instanceName, - String fromState, String toState, List pendingMessages) { - - for (Message message : pendingMessages) { - if (message.getResourceName().equals(resourceName) - && message.getPartitionName().equals(partition.getPartitionName()) - && message.getTgtName().equals(instanceName) && message.getFromState().equals(fromState) - && message.getToState().equals(toState)) { - return true; - } - } - - return false; - } - - /** - * Check if a state transition is upward - * @param fromState The from state - * @param toState The to state - * @param stateModelDef The state model definition - * @return True if it's an upward state transition, false otherwise - */ - private boolean isUpwardStateTransition(String fromState, String toState, - StateModelDefinition stateModelDef) { - - if (fromState == null || toState == null) { - return false; - } - - Map statePriorityMap = stateModelDef.getStatePriorityMap(); - - Integer fromStateWeight = statePriorityMap.get(fromState); - Integer toStateWeight = statePriorityMap.get(toState); - - if (fromStateWeight == null || toStateWeight == null) { - return false; - } - - return toStateWeight < fromStateWeight; - } - /** * Get pending upward state transition messages from non-second top state to second top or top * state @@ -395,7 +342,7 @@ private boolean isUpwardStateTransition(String fromState, String toState, * @param stateModelDef The state model definition * @return List of pending messages for upward state transitions */ - private List getPendingUpwardStateTransitionMessages(String resourceName, + private List getPendingTransitionsToTopOrSecondTopStates(String resourceName, Partition partition, CurrentStateOutput currentStateOutput, StateModelDefinition stateModelDef) { List pendingUpwardSTMessages = new ArrayList<>(); @@ -411,7 +358,7 @@ private List getPendingUpwardStateTransitionMessages(String resourceNam // Check if it's an upward state transition from non-second top state to second top or top // state - if (isUpwardStateTransition(fromState, toState, stateModelDef) + if (stateModelDef.isUpwardStateTransition(fromState, toState) && !isSecondTopState(fromState, stateModelDef) && (isSecondTopState(toState, stateModelDef) || isTopState(toState, stateModelDef))) { pendingUpwardSTMessages.add(message); diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java index 8a15452ea5..8555912497 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Message.java +++ b/helix-core/src/main/java/org/apache/helix/model/Message.java @@ -138,12 +138,8 @@ public enum STRebalanceType { /** * Compares the creation time of two Messages */ - public static final Comparator CREATE_TIME_COMPARATOR = new Comparator() { - @Override - public int compare(Message m1, Message m2) { - return new Long(m1.getCreateTimeStamp()).compareTo(new Long(m2.getCreateTimeStamp())); - } - }; + public static final Comparator CREATE_TIME_COMPARATOR = + (m1, m2) -> Long.compare(m2.getCreateTimeStamp(), m1.getCreateTimeStamp()); /** * Instantiate a message @@ -937,8 +933,14 @@ public void setSrcClusterName(String clusterName) { } /** - * Set the currentReplicaNumber for transition-related messages - * @param currentReplicaNumber the replica count + * Set replica number for participant-side message prioritization. + * This field indicates the number of top and second-top state replicas at the time a state transition message is generated. + * It is used to prioritize messages, with lower values indicating higher priority. + * Participants can use this in custom thread pools or message handlers to process + * critical transitions first during recovery scenarios. + * Default value is -1 for transitions that don't require prioritization. + * @param currentReplicaNumber the replica priority number (-1 for no prioritization, >=0 for + * prioritized) */ public void setCurrentReplicaNumber(int currentReplicaNumber) { _record.setIntField(Attributes.CURRENT_REPLICA_NUMBER.name(), currentReplicaNumber); diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java index fcf24fb305..becccfa943 100644 --- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java +++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java @@ -505,4 +505,28 @@ public static Map getStateCounts(Map stateMap) } return stateCounts; } + + /** + * Check if a state transition is upward + * @param fromState source state + * @param toState destination state + * @return True if it's an upward state transition, false otherwise + */ + public boolean isUpwardStateTransition(String fromState, String toState) { + + if (fromState == null || toState == null) { + return false; + } + + Map statePriorityMap = getStatePriorityMap(); + + Integer fromStateWeight = statePriorityMap.get(fromState); + Integer toStateWeight = statePriorityMap.get(toState); + + if (fromStateWeight == null || toStateWeight == null) { + return false; + } + + return toStateWeight < fromStateWeight; + } } diff --git a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java index a82c1d58ba..acacf1ab97 100644 --- a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java +++ b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java @@ -48,7 +48,7 @@ public static Message createStateTransitionCancellationMessage(String srcInstanc toState); Message message = - createStateTransitionMessage(Message.MessageType.STATE_TRANSITION_CANCELLATION, + createBasicStateTransitionMessage(Message.MessageType.STATE_TRANSITION_CANCELLATION, srcInstanceName, srcSessionId, resource, partitionName, instanceName, currentState, nextState, sessionId, stateModelDefName); @@ -60,28 +60,6 @@ public static Message createStateTransitionCancellationMessage(String srcInstanc return null; } - public static Message createStateTransitionMessage(String srcInstanceName, String srcSessionId, - Resource resource, String partitionName, String instanceName, String currentState, - String nextState, String tgtSessionId, String stateModelDefName) { - Message message = - createStateTransitionMessage(Message.MessageType.STATE_TRANSITION, srcInstanceName, - srcSessionId, resource, partitionName, instanceName, currentState, nextState, tgtSessionId, - stateModelDefName); - - // Set the retry count for state transition messages. - // TODO: make the retry count configurable in ClusterConfig or IdealState - message.setRetryCount(DEFAULT_STATE_TRANSITION_MESSAGE_RETRY_COUNT); - - if (resource.getResourceGroupName() != null) { - message.setResourceGroupName(resource.getResourceGroupName()); - } - if (resource.getResourceTag() != null) { - message.setResourceTag(resource.getResourceTag()); - } - - return message; - } - /** * Creates a message to change participant status * {@link org.apache.helix.model.LiveInstance.LiveInstanceStatus} @@ -121,7 +99,7 @@ private static Message createBasicMessage(Message.MessageType messageType, Strin } /* Creates state transition or state transition cancellation message */ - private static Message createStateTransitionMessage(Message.MessageType messageType, + private static Message createBasicStateTransitionMessage(Message.MessageType messageType, String srcInstanceName, String srcSessionId, Resource resource, String partitionName, String instanceName, String currentState, String nextState, String tgtSessionId, String stateModelDefName) { @@ -138,30 +116,62 @@ private static Message createStateTransitionMessage(Message.MessageType messageT } /** - * Create a state transition message with currentReplicaNumber for prioritization - * @param msgSender message sender - * @param sessionId session id + * Create a state transition message with replica prioritization metadata + * @param srcInstanceName source instance name + * @param srcSessionId source session id * @param resource resource * @param partitionName partition name - * @param instanceName instance name - * @param fromState from state - * @param toState to state - * @param sessionIdForInstance session id for instance - * @param stateModelDefName state model def name - * @param currentReplicaNumber the current replica number (for prioritization) - * @return message + * @param instanceName target instance name + * @param currentState current state + * @param nextState next state + * @param tgtSessionId target session id + * @param stateModelDefName state model definition name + * @param currentReplicaNumber replica priority number (-1 for no prioritization, >=0 for + * prioritized) + * @return state transition message */ - public static Message createStateTransitionMessage(String msgSender, String sessionId, - Resource resource, String partitionName, String instanceName, String fromState, - String toState, String sessionIdForInstance, String stateModelDefName, - int currentReplicaNumber) { + public static Message createStateTransitionMessage(String srcInstanceName, String srcSessionId, + Resource resource, String partitionName, String instanceName, String currentState, + String nextState, String tgtSessionId, String stateModelDefName, int currentReplicaNumber) { + Message message = createBasicStateTransitionMessage(Message.MessageType.STATE_TRANSITION, + srcInstanceName, srcSessionId, resource, partitionName, instanceName, currentState, + nextState, tgtSessionId, stateModelDefName); + + // Set the retry count for state transition messages. + // TODO: make the retry count configurable in ClusterConfig or IdealState + message.setRetryCount(DEFAULT_STATE_TRANSITION_MESSAGE_RETRY_COUNT); - Message message = createStateTransitionMessage(msgSender, sessionId, resource, partitionName, - instanceName, fromState, toState, sessionIdForInstance, stateModelDefName); + if (resource.getResourceGroupName() != null) { + message.setResourceGroupName(resource.getResourceGroupName()); + } + if (resource.getResourceTag() != null) { + message.setResourceTag(resource.getResourceTag()); + } - // Set the current replica number for prioritization + // Set replica number for participant-side prioritization message.setCurrentReplicaNumber(currentReplicaNumber); return message; } + + /** + * Create a state transition message (backward compatibility) + * @param srcInstanceName source instance name + * @param srcSessionId source session id + * @param resource resource + * @param partitionName partition name + * @param instanceName target instance name + * @param currentState current state + * @param nextState next state + * @param tgtSessionId target session id + * @param stateModelDefName state model definition name + * @return state transition message + */ + public static Message createStateTransitionMessage(String srcInstanceName, String srcSessionId, + Resource resource, String partitionName, String instanceName, String currentState, + String nextState, String tgtSessionId, String stateModelDefName) { + // currentReplicaNumber is set to -1 for ST messages needing no prioritization metadata (backward compatibility) + return createStateTransitionMessage(srcInstanceName, srcSessionId, resource, partitionName, + instanceName, currentState, nextState, tgtSessionId, stateModelDefName, -1); + } } From ffb3dd706c1ed141702ccc450e7e1142e67bbf2d Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Thu, 5 Jun 2025 19:01:17 -0700 Subject: [PATCH 03/11] updated targetActiveReplicaCount and related tests, expanded on comments, renamed variables. --- .../stages/MessageGenerationPhase.java | 59 +++++++++---- .../TestPrioritizationMessageGeneration.java | 87 +++++++++++++++++-- 2 files changed, 124 insertions(+), 22 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index e984f5bbc6..323ab4bf54 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -163,15 +163,22 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr // desired-state->list of generated-messages Map> messageMap = new HashMap<>(); - // Calculate metadata for message prioritization - int partitionInstanceCount = instanceStateMap.keySet().size(); - - // Get pending upward state transition messages to second top or top state - List pendingUpwardStateTransitionMessagesToTopOrSecondTopStates = getPendingTransitionsToTopOrSecondTopStates( + // Calculate target active replica count after convergence (excluding OFFLINE and DROPPED replicas) + // This counts replicas that will be actively serving traffic after all state transitions complete. + // For example, Includes: MASTER, SLAVE, and other serving states (excludes OFFLINE, DROPPED, NO_DESIRED_STATE) + int targetActiveReplicaCount = calculateTargetActiveReplicaCount(instanceStateMap, stateModelDef); + + // Get pending upward state transition messages to second top or top state. + // This holds good for state models with no more than 3 state in the state machine definition + List pendingUpMsgToTopOrSecondTop = getPendingTransitionsToTopOrSecondTopStates( resourceName, partition, currentStateOutput, stateModelDef); - // Initialize replica counter for prioritization - int initialReplicaNumber = partitionInstanceCount - pendingUpwardStateTransitionMessagesToTopOrSecondTopStates.size(); + // Calculate remaining replica positions available for new upward state transitions + // This represents how many more upward transitions can be assigned replica numbers + // This counter decrements from (targetActiveReplicaCount - pendingUpMsgToTopOrSecondTop) down to 0 + // ensuring first replica gets highest number for proper recovery ordering + // Example: targetActiveReplicaCount=3, pendingUpMsgToTopOrSecondTop=1 → counter starts at 2, assigns replica numbers 2,1,0 + int replicaNumberCounter = targetActiveReplicaCount - pendingUpMsgToTopOrSecondTop.size(); for (String instanceName : instanceStateMap.keySet()) { @@ -260,17 +267,18 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr pendingMessage, manager, resource, partition, sessionIdMap, instanceName, stateModelDef, cancellationMessage, isCancellationEnabled); } else { - // Default currentReplicaNumber is -1 (provides metadata for participant-side prioritization) - int currentReplicaNumber = -1; + // Set currentReplicaNumber to provide metadata for potential message prioritization by participant + int currentReplicaNumber = -1; // -1 by default + // Check if this is an upward state transition from non-second top state to second top // or top state if (stateModelDef.isUpwardStateTransition(currentState, nextState) && !stateModelDef.getSecondTopStates().contains(currentState) && (isSecondTopState(nextState, stateModelDef) - || isTopState(nextState, stateModelDef))) { + || stateModelDef.getTopState().contains(nextState))) { // Assign the replica number for prioritization - currentReplicaNumber = initialReplicaNumber--; + currentReplicaNumber = replicaNumberCounter--; } // Create new state transition message @@ -314,13 +322,30 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr } /** - * Check if a state is a top state - * @param state The state to check + * Calculate the target active replica count after state convergence. + * This method counts only replicas that will be actively serving traffic after all + * state transitions complete. It excludes replicas being dropped or in inactive states. + * + * @param instanceStateMap Map of instance to desired state * @param stateModelDef The state model definition - * @return True if it's a top state, false otherwise + * @return Number of active replicas after convergence */ - private boolean isTopState(String state, StateModelDefinition stateModelDef) { - return stateModelDef.getTopState().contains(state); + private int calculateTargetActiveReplicaCount(Map instanceStateMap, + StateModelDefinition stateModelDef) { + int targetActiveReplicaCount = 0; + String initialState = stateModelDef.getInitialState(); + + for (String desiredState : instanceStateMap.values()) { + // Count potential active replicas after convergence (not initial state/dropped state) + // Note : calling it potential as counting active replicas after convergence is hard as upward state transitions are in progress too. + if (desiredState != null && !desiredState.equals(initialState) + && !desiredState.equals(NO_DESIRED_STATE) + && !desiredState.equals(HelixDefinedState.DROPPED.name())) { + targetActiveReplicaCount++; + } + } + + return targetActiveReplicaCount; } /** @@ -360,7 +385,7 @@ private List getPendingTransitionsToTopOrSecondTopStates(String resourc // state if (stateModelDef.isUpwardStateTransition(fromState, toState) && !isSecondTopState(fromState, stateModelDef) - && (isSecondTopState(toState, stateModelDef) || isTopState(toState, stateModelDef))) { + && (isSecondTopState(toState, stateModelDef) || stateModelDef.getTopState().contains(toState))) { pendingUpwardSTMessages.add(message); } } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index 9c16c0674c..5ecda09886 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -101,7 +101,7 @@ public void testPrioritizationForUpwardStateTransition() throws Exception { INSTANCE_2, INSTANCE_3 }) { Assert.assertTrue(replicaNumbers.get(instance) >= 0, - "Replica number should be assigned for " + instance); + "Replica number should be assigned for upward state transition" + instance); } // Verify the replica numbers are decreasing (higher number = higher priority) @@ -155,7 +155,7 @@ public void testPrioritizationForUpwardStateTransitionWithAllOfflineInstances() INSTANCE_0, INSTANCE_1, INSTANCE_2 }) { Assert.assertTrue(replicaNumbers.get(instance) >= 0, - "Replica number should be assigned for " + instance); + "Replica number should be assigned for upward state transition" + instance); } // Verify the replica numbers are decreasing (higher number = higher priority) @@ -198,7 +198,7 @@ public void testNoPrioritizationForNonUpwardTransitions() throws Exception { // Should have default replica number (-1) for non-upward transition Message msg = messages.get(0); Assert.assertEquals(msg.getCurrentReplicaNumber(), -1, - "Non-upward transitions should not be prioritized"); + "Non-upward state transitions should not have replica numbers assigned"); } @Test @@ -252,9 +252,11 @@ public void testPrioritizationWithPendingMessages() throws Exception { Assert.assertEquals(messages.size(), 1); Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_1); - // The message should have a replica number since pending messages are considered + // The current replica number should account for the pending message + // targetActiveReplicaCount = 2 (2 SLAVE), pendingUpwardMessages = 1 + // replicaNumberCounter = 2 - 1 = 1, so new message gets current replica number 1 Assert.assertTrue(messages.get(0).getCurrentReplicaNumber() >= 0, - "Should assign replica number considering pending messages"); + "Should assign current replica number considering pending upward messages"); } @Test @@ -295,6 +297,81 @@ public void testTransitionFromSecondTopState() throws Exception { "Transitions from second top states should not be prioritized"); } + @Test + public void testReplicaCountCalculationWithDroppedReplicas() throws Exception { + // Test that DROPPED replicas are excluded from target active replica count + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Set up current states + Map currentStateMap = new HashMap<>(); + + // Instance 0: MASTER -> will stay MASTER + CurrentState currentState0 = new CurrentState(TEST_RESOURCE); + currentState0.setState(PARTITION_0, "MASTER"); + currentState0.setSessionId(SESSION_ID); + currentState0.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, currentState0); + + // Instance 1: SLAVE -> will be DROPPED + CurrentState currentState1 = new CurrentState(TEST_RESOURCE); + currentState1.setState(PARTITION_0, "SLAVE"); + currentState1.setSessionId(SESSION_ID); + currentState1.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_1, currentState1); + + // Instance 2: OFFLINE -> will become SLAVE + CurrentState currentState2 = new CurrentState(TEST_RESOURCE); + currentState2.setState(PARTITION_0, "OFFLINE"); + currentState2.setSessionId(SESSION_ID); + currentState2.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_2, currentState2); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state: MASTER stays, one SLAVE gets DROPPED, one OFFLINE becomes SLAVE + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "MASTER"); // No change + partitionMap.put(INSTANCE_1, "DROPPED"); // SLAVE -> DROPPED (excluded from active count) + partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 2 messages: SLAVE->DROPPED and OFFLINE->SLAVE + Assert.assertEquals(messages.size(), 2); + + Message upwardTransitionMsg = null; + Message droppedTransitionMsg = null; + for (Message msg : messages) { + if (msg.getTgtName().equals(INSTANCE_2)) { + upwardTransitionMsg = msg; + } else if (msg.getTgtName().equals(INSTANCE_1)) { + droppedTransitionMsg = msg; + } + } + + Assert.assertNotNull(upwardTransitionMsg, "Should have upward state transition message"); + Assert.assertNotNull(droppedTransitionMsg, "Should have dropped state transition message"); + + // Upward transition should get replica number + // targetActiveReplicaCount = 2 (1 MASTER + 1 SLAVE, DROPPED excluded) + // replicaNumberCounter = 2 - 0 = 2, upward transition gets current replica number 2 + Assert.assertEquals(upwardTransitionMsg.getCurrentReplicaNumber(), 2, + "Upward state transition message should have current replica number assigned based on active replica count excluding DROPPED"); + + // DROPPED transition should not get current replica number (it's a downward transition anyway) + Assert.assertEquals(droppedTransitionMsg.getCurrentReplicaNumber(), -1, + "DROPPED state transition message should not have current replica number assigned."); + } + private ClusterEvent prepareClusterEvent(StateModelDefinition stateModelDef, CurrentStateOutput currentStateOutput) { ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); From a283f1824f1665e8de5cf33f2b441b6b797c27bd Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Thu, 12 Jun 2025 09:49:12 -0700 Subject: [PATCH 04/11] Update logic to use currentActiveReplicaNumber --- .../stages/MessageGenerationPhase.java | 180 ++++---- .../java/org/apache/helix/model/Message.java | 41 +- .../org/apache/helix/util/MessageUtil.java | 20 +- .../TestPrioritizationMessageGeneration.java | 432 ++++++++++++++---- 4 files changed, 463 insertions(+), 210 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 323ab4bf54..81505b83b4 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -163,22 +163,31 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr // desired-state->list of generated-messages Map> messageMap = new HashMap<>(); - // Calculate target active replica count after convergence (excluding OFFLINE and DROPPED replicas) - // This counts replicas that will be actively serving traffic after all state transitions complete. - // For example, Includes: MASTER, SLAVE, and other serving states (excludes OFFLINE, DROPPED, NO_DESIRED_STATE) - int targetActiveReplicaCount = calculateTargetActiveReplicaCount(instanceStateMap, stateModelDef); - - // Get pending upward state transition messages to second top or top state. - // This holds good for state models with no more than 3 state in the state machine definition - List pendingUpMsgToTopOrSecondTop = getPendingTransitionsToTopOrSecondTopStates( - resourceName, partition, currentStateOutput, stateModelDef); - - // Calculate remaining replica positions available for new upward state transitions - // This represents how many more upward transitions can be assigned replica numbers - // This counter decrements from (targetActiveReplicaCount - pendingUpMsgToTopOrSecondTop) down to 0 - // ensuring first replica gets highest number for proper recovery ordering - // Example: targetActiveReplicaCount=3, pendingUpMsgToTopOrSecondTop=1 → counter starts at 2, assigns replica numbers 2,1,0 - int replicaNumberCounter = targetActiveReplicaCount - pendingUpMsgToTopOrSecondTop.size(); + /** + * Calculate the current active replica count based on state model type. + * This represents the number of replicas currently serving traffic for this partition + * Active replicas include: top states, secondary top states (for single-top state models), + * and ERROR states. + * All qualifying state transitions for this partition will receive this same value, + * allowing clients to understand the current availability level and prioritize accordingly. + */ + int currentActiveReplicaCount; + if (stateModelDef.isSingleTopStateModel()) { + // For single-top state models (e.g., OFFLINE→STANDBY→ONLINE) + // Count replicas in top state, secondary top state and ERROR state + currentActiveReplicaCount = (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) + || stateModelDef.getSecondTopStates().contains(state) + || HelixDefinedState.ERROR.name().equals(state)) + .count(); + } else { + // For multi-top state models (e.g., OFFLINE→ONLINE) + // Count replicas that will be in top state and ERROR state + currentActiveReplicaCount = (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) + || HelixDefinedState.ERROR.name().equals(state)) + .count(); + } for (String instanceName : instanceStateMap.keySet()) { @@ -267,31 +276,42 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr pendingMessage, manager, resource, partition, sessionIdMap, instanceName, stateModelDef, cancellationMessage, isCancellationEnabled); } else { - // Set currentReplicaNumber to provide metadata for potential message prioritization by participant - int currentReplicaNumber = -1; // -1 by default - - // Check if this is an upward state transition from non-second top state to second top - // or top state + // Set currentActiveReplicaNumber to provide metadata for potential message prioritization by + // participant. + // Assign the current active replica count to all qualifying upward transitions for this + // partition. + // This ensures consistent prioritization metadata across concurrent state transitions. + int currentActiveReplicaNumber = -1; // -1 indicates no prioritization metadata, for eg: + // Downward ST messages get a -1. + + /** + * Assign currentActiveReplicaNumber for qualifying upward state transitions. + * Criteria for assignment: + * - Must be an upward state transition according to state model + * - Current state must not be considered active (according to state model type) + * - Target state must be considered active (according to state model type) + */ if (stateModelDef.isUpwardStateTransition(currentState, nextState) - && !stateModelDef.getSecondTopStates().contains(currentState) - && (isSecondTopState(nextState, stateModelDef) - || stateModelDef.getTopState().contains(nextState))) { + && !isCurrentlyActive(currentState, stateModelDef, + stateModelDef.isSingleTopStateModel()) + && isTargetActive(nextState, stateModelDef, + stateModelDef.isSingleTopStateModel())) { - // Assign the replica number for prioritization - currentReplicaNumber = replicaNumberCounter--; + // All qualifying transitions for this partition get the same currentActiveReplicaNumber + currentActiveReplicaNumber = currentActiveReplicaCount; } // Create new state transition message message = MessageUtil .createStateTransitionMessage(manager.getInstanceName(), manager.getSessionId(), resource, partition.getPartitionName(), instanceName, currentState, nextState, - sessionIdMap.get(instanceName), stateModelDef.getId(), currentReplicaNumber); + sessionIdMap.get(instanceName), stateModelDef.getId(), currentActiveReplicaNumber); if (logger.isDebugEnabled()) { LogUtil.logDebug(logger, _eventId, String.format( - "Resource %s partition %s for instance %s with currentState %s, nextState %s and currentReplicaNumber %d", + "Resource %s partition %s for instance %s with currentState %s, nextState %s and currentActiveReplicaNumber %d", resource.getResourceName(), partition.getPartitionName(), instanceName, - currentState, nextState, currentReplicaNumber)); + currentState, nextState, currentActiveReplicaNumber)); } } } @@ -322,79 +342,53 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr } /** - * Calculate the target active replica count after state convergence. - * This method counts only replicas that will be actively serving traffic after all - * state transitions complete. It excludes replicas being dropped or in inactive states. - * - * @param instanceStateMap Map of instance to desired state - * @param stateModelDef The state model definition - * @return Number of active replicas after convergence + * Determines if the given current state is considered active based on the state model type. + * For single-top state models, top, secondary top, and ERROR states are active. + * For multi-top state models, top and ERROR states are active. + * ERROR state replicas are considered active in HELIX as they do not affect availability. + * @param currentState The current state to check + * @param stateModelDef State model definition containing state hierarchy information + * @param isSingleTopState Whether this is a single-top state model + * @return true if the current state is considered active, false otherwise */ - private int calculateTargetActiveReplicaCount(Map instanceStateMap, - StateModelDefinition stateModelDef) { - int targetActiveReplicaCount = 0; - String initialState = stateModelDef.getInitialState(); - - for (String desiredState : instanceStateMap.values()) { - // Count potential active replicas after convergence (not initial state/dropped state) - // Note : calling it potential as counting active replicas after convergence is hard as upward state transitions are in progress too. - if (desiredState != null && !desiredState.equals(initialState) - && !desiredState.equals(NO_DESIRED_STATE) - && !desiredState.equals(HelixDefinedState.DROPPED.name())) { - targetActiveReplicaCount++; - } + private boolean isCurrentlyActive(String currentState, StateModelDefinition stateModelDef, + boolean isSingleTopState) { + // ERROR state is always considered active regardless of state model type + if (HelixDefinedState.ERROR.name().equals(currentState)) { + return true; + } + if (isSingleTopState) { + return stateModelDef.getTopState().contains(currentState) + || stateModelDef.getSecondTopStates().contains(currentState); + } else { + return stateModelDef.getTopState().contains(currentState); } - - return targetActiveReplicaCount; - } - - /** - * Check if a state is a second top state - * @param state The state to check - * @param stateModelDef The state model definition - * @return True if it's a second top state, false otherwise - */ - private boolean isSecondTopState(String state, StateModelDefinition stateModelDef) { - return stateModelDef.getSecondTopStates().contains(state); } /** - * Get pending upward state transition messages from non-second top state to second top or top - * state - * @param resourceName The resource name - * @param partition The partition - * @param currentStateOutput The current state output - * @param stateModelDef The state model definition - * @return List of pending messages for upward state transitions + * Determines if the given target state is considered active based on the state model type. + * For single-top state models, both top,secondary top and ERROR states are active. + * For multi-top state models, top and ERROR states are active. + * @param targetState The target state to check + * @param stateModelDef State model definition containing state hierarchy information + * @param isSingleTopState Whether this is a single-top state model + * @return true if the target state is considered active, false otherwise */ - private List getPendingTransitionsToTopOrSecondTopStates(String resourceName, - Partition partition, CurrentStateOutput currentStateOutput, - StateModelDefinition stateModelDef) { - List pendingUpwardSTMessages = new ArrayList<>(); - - // Instance -> PendingMessage - Map pendingMessages = - currentStateOutput.getPendingMessageMap(resourceName, partition); - - if (pendingMessages != null && !pendingMessages.isEmpty()) { - for (Message message : pendingMessages.values()) { - String fromState = message.getFromState(); - String toState = message.getToState(); - - // Check if it's an upward state transition from non-second top state to second top or top - // state - if (stateModelDef.isUpwardStateTransition(fromState, toState) - && !isSecondTopState(fromState, stateModelDef) - && (isSecondTopState(toState, stateModelDef) || stateModelDef.getTopState().contains(toState))) { - pendingUpwardSTMessages.add(message); - } - } + private boolean isTargetActive(String targetState, StateModelDefinition stateModelDef, + boolean isSingleTopState) { + // ERROR state is always considered active regardless of state model type + if (HelixDefinedState.ERROR.name().equals(targetState)) { + return true; + } + if (isSingleTopState) { + return stateModelDef.getTopState().contains(targetState) + || stateModelDef.getSecondTopStates().contains(targetState); + } else { + return stateModelDef.getTopState().contains(targetState); } - - return pendingUpwardSTMessages; } - private boolean shouldCreateSTCancellation(Message pendingMessage, String desiredState, + private boolean shouldCreateSTCancellation(Message pendingMessage, String desiredState, String initialState) { if (pendingMessage == null) { return false; diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java index 8555912497..a2a0c7be99 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Message.java +++ b/helix-core/src/main/java/org/apache/helix/model/Message.java @@ -105,7 +105,7 @@ public enum Attributes { EXPIRY_PERIOD, SRC_CLUSTER, ST_REBALANCE_TYPE, - CURRENT_REPLICA_NUMBER + CURRENT_ACTIVE_REPLICA_NUMBER } /** @@ -933,25 +933,36 @@ public void setSrcClusterName(String clusterName) { } /** - * Set replica number for participant-side message prioritization. - * This field indicates the number of top and second-top state replicas at the time a state transition message is generated. - * It is used to prioritize messages, with lower values indicating higher priority. - * Participants can use this in custom thread pools or message handlers to process - * critical transitions first during recovery scenarios. - * Default value is -1 for transitions that don't require prioritization. - * @param currentReplicaNumber the replica priority number (-1 for no prioritization, >=0 for - * prioritized) + * Set current active replica count for participant-side message prioritization. + * This field indicates the number of replicas currently in active states (including ERROR states) + * for this partition at the time the state transition message is generated. + * Active states include top states, secondary top states (for single-top state models), + * and ERROR states. + * + * This metadata enables participants to prioritize recovery scenarios (low active counts) + * over load balancing scenarios (high active counts) in custom thread pools or message handlers. + * For example, 2→3 transitions get higher priority than 3→4 transitions. + * + * Default value is -1 for transitions that don't require prioritization metadata.(for eg : downward transitions). + * + * @param currentActiveReplicaNumber the number of currently active replicas (-1 when there is no prioritization metadata, + * >=0 for transitions containing current availability level) */ - public void setCurrentReplicaNumber(int currentReplicaNumber) { - _record.setIntField(Attributes.CURRENT_REPLICA_NUMBER.name(), currentReplicaNumber); + public void setCurrentActiveReplicaNumber(int currentActiveReplicaNumber) { + _record.setIntField(Attributes.CURRENT_ACTIVE_REPLICA_NUMBER.name(), currentActiveReplicaNumber); } /** - * Get the "CurrentReplicaNumber" for transition-related messages - * @return CurrentReplicaNumber, or null for other message types + * Get the current active replica count for this partition at message generation time. + * This value represents the number of replicas in active states (including ERROR states) before + * any state transitions occur, enabling participant-side message prioritization based on + * current availability levels. + * @return current active replica count, or -1 for cases where we don't provide metadata for + * prioritization like downward state transitions. */ - public int getCurrentReplicaNumber() { - return _record.getIntField(Attributes.CURRENT_REPLICA_NUMBER.name(), -1); + + public int getCurrentActiveReplicaNumber() { + return _record.getIntField(Attributes.CURRENT_ACTIVE_REPLICA_NUMBER.name(), -1); } /** diff --git a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java index acacf1ab97..4a93050dcb 100644 --- a/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java +++ b/helix-core/src/main/java/org/apache/helix/util/MessageUtil.java @@ -126,13 +126,20 @@ private static Message createBasicStateTransitionMessage(Message.MessageType mes * @param nextState next state * @param tgtSessionId target session id * @param stateModelDefName state model definition name - * @param currentReplicaNumber replica priority number (-1 for no prioritization, >=0 for - * prioritized) + * @param currentActiveReplicaNumber The number of replicas currently in active states + * for this partition before any state transitions occur. This metadata + * enables participant-side message prioritization by indicating the + * current availability level (e.g., 0→1 recovery scenarios get higher + * priority than 2→3 load balancing scenarios). Set to -1 for transitions + * that should not be prioritized (downward transitions). + * Active states include top states, secondary top states (for single-top + * state models), and ERROR states since they are still considered active by Helix. * @return state transition message */ public static Message createStateTransitionMessage(String srcInstanceName, String srcSessionId, Resource resource, String partitionName, String instanceName, String currentState, - String nextState, String tgtSessionId, String stateModelDefName, int currentReplicaNumber) { + String nextState, String tgtSessionId, String stateModelDefName, + int currentActiveReplicaNumber) { Message message = createBasicStateTransitionMessage(Message.MessageType.STATE_TRANSITION, srcInstanceName, srcSessionId, resource, partitionName, instanceName, currentState, nextState, tgtSessionId, stateModelDefName); @@ -148,8 +155,8 @@ public static Message createStateTransitionMessage(String srcInstanceName, Strin message.setResourceTag(resource.getResourceTag()); } - // Set replica number for participant-side prioritization - message.setCurrentReplicaNumber(currentReplicaNumber); + // Set current active replica number for participant-side prioritization + message.setCurrentActiveReplicaNumber(currentActiveReplicaNumber); return message; } @@ -170,7 +177,8 @@ public static Message createStateTransitionMessage(String srcInstanceName, Strin public static Message createStateTransitionMessage(String srcInstanceName, String srcSessionId, Resource resource, String partitionName, String instanceName, String currentState, String nextState, String tgtSessionId, String stateModelDefName) { - // currentReplicaNumber is set to -1 for ST messages needing no prioritization metadata (backward compatibility) + // currentActiveReplicaNumber is set to -1 for ST messages needing no prioritization metadata + // (backward compatibility) return createStateTransitionMessage(srcInstanceName, srcSessionId, resource, partitionName, instanceName, currentState, nextState, tgtSessionId, stateModelDefName, -1); } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index 5ecda09886..fc81d1545b 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -54,7 +54,7 @@ public class TestPrioritizationMessageGeneration extends MessageGenerationPhase private static final String SESSION_ID = "123"; @Test - public void testPrioritizationForUpwardStateTransition() throws Exception { + public void testCurrentReplicaCountForUpwardTransitions() throws Exception { // Test prioritization for upward state transitions from non-second top states // to second top or top states @@ -89,43 +89,29 @@ public void testPrioritizationForUpwardStateTransition() throws Exception { // Should generate 2 messages Assert.assertEquals(messages.size(), 2); - // Messages should have replica numbers assigned based on priority - Map replicaNumbers = new HashMap<>(); + // Both messages should have the same currentActiveReplicaNumber = current active replica count + // Current active replicas: 1 MASTER + 1 SLAVE = 2 for (Message msg : messages) { - int replicaNumber = msg.getCurrentReplicaNumber(); - replicaNumbers.put(msg.getTgtName(), replicaNumber); + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 2, + "All upward transitions should have currentActiveReplicaNumber = current active replica count (2)"); + Assert.assertTrue(msg.getTgtName().equals(INSTANCE_2) || msg.getTgtName().equals(INSTANCE_3), + "Messages should be for instances transitioning from OFFLINE to SLAVE"); } - - // Verify replica numbers are assigned (not -1) for upward transitions - for (String instance : new String[] { - INSTANCE_2, INSTANCE_3 - }) { - Assert.assertTrue(replicaNumbers.get(instance) >= 0, - "Replica number should be assigned for upward state transition" + instance); - } - - // Verify the replica numbers are decreasing (higher number = higher priority) - Assert.assertTrue(replicaNumbers.get(INSTANCE_3) > replicaNumbers.get(INSTANCE_2), - "Earlier transitions should have higher replica numbers"); } @Test - public void testPrioritizationForUpwardStateTransitionWithAllOfflineInstances() throws Exception { - - // Test prioritization for upward state transitions from non-second top states - // to second top or top states + public void testZeroReplicaScenario() throws Exception { + // Test scenario with 0 current active replicas (all OFFLINE) StateModelDefinition stateModelDef = new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - // Set up current states - multiple instances in OFFLINE state - // We want to test that they get prioritized when transitioning to SLAVE (second top) or MASTER - // (top) + // Current state: All instances in OFFLINE state Map currentStateMap = setupCurrentStatesForPrioritizationCase2(); CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - // Best possible state wants to move some nodes to SLAVE/MASTER + // Best possible state wants to create 1 MASTER, 2 SLAVE Map partitionMap = new HashMap<>(); partitionMap.put(INSTANCE_0, "SLAVE"); // Offline -> Slave (upward to second top) partitionMap.put(INSTANCE_1, "MASTER"); // Offline -> Master (upward to top) @@ -143,33 +129,20 @@ public void testPrioritizationForUpwardStateTransitionWithAllOfflineInstances() // Should generate 3 messages Assert.assertEquals(messages.size(), 3); - // Messages should have replica numbers assigned based on priority - Map replicaNumbers = new HashMap<>(); + // All messages should have currentActiveReplicaNumber = 0 (current active replica count) for (Message msg : messages) { - int replicaNumber = msg.getCurrentReplicaNumber(); - replicaNumbers.put(msg.getTgtName(), replicaNumber); - } - - // Verify replica numbers are assigned (not -1) for upward transitions - for (String instance : new String[] { - INSTANCE_0, INSTANCE_1, INSTANCE_2 - }) { - Assert.assertTrue(replicaNumbers.get(instance) >= 0, - "Replica number should be assigned for upward state transition" + instance); + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 0, + "All upward transitions should have currentActiveReplicaNumber = 0"); } - - // Verify the replica numbers are decreasing (higher number = higher priority) - Assert.assertTrue(replicaNumbers.get(INSTANCE_2) > replicaNumbers.get(INSTANCE_0), - "Earlier transitions should have higher replica numbers"); } @Test - public void testNoPrioritizationForNonUpwardTransitions() throws Exception { - // Test that non-upward transitions don't get prioritized + public void testNoReplicaNumberForNonUpwardTransitions() throws Exception { + // Test that non-upward transitions don't get currentActiveReplicaNumber assigned StateModelDefinition stateModelDef = new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - // Set up current states - instance already in SLAVE (second top) + // Current state: Instance in SLAVE state Map currentStateMap = new HashMap<>(); CurrentState currentState = new CurrentState(TEST_RESOURCE); currentState.setState(PARTITION_0, "SLAVE"); @@ -195,49 +168,93 @@ public void testNoPrioritizationForNonUpwardTransitions() throws Exception { Assert.assertEquals(messages.size(), 1); - // Should have default replica number (-1) for non-upward transition + // Should have default currentActiveReplicaNumber (-1) for non-upward transition Message msg = messages.get(0); - Assert.assertEquals(msg.getCurrentReplicaNumber(), -1, - "Non-upward state transitions should not have replica numbers assigned"); + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, + "Non-upward state transitions should not have currentActiveReplicaNumber assigned"); } @Test - public void testPrioritizationWithPendingMessages() throws Exception { - // Test prioritization when there are already pending messages + public void testNoReplicaNumberForSecondTopToTopTransitions() throws Exception { + // Tests No currentActiveReplicaNumber assigned for Second Top -> Top State Transitions for Single-Top-State + // State Model StateModelDefinition stateModelDef = new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // Current state: Instance in SLAVE state (second top) Map currentStateMap = new HashMap<>(); + CurrentState currentState = new CurrentState(TEST_RESOURCE); + currentState.setState(PARTITION_0, "SLAVE"); + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, currentState); - // Instance 0: OFFLINE with pending OFFLINE->SLAVE message - CurrentState currentState0 = new CurrentState(TEST_RESOURCE); - currentState0.setState(PARTITION_0, "OFFLINE"); - currentState0.setSessionId(SESSION_ID); - currentState0.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, currentState0); + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - // Create pending message separately - Message pendingMsg = createMessage("OFFLINE", "SLAVE", INSTANCE_0); - pendingMsg.setMsgId(UUID.randomUUID().toString()); + // Best possible state wants to promote to MASTER (second top -> top transition) + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "MASTER"); - // Instance 1: OFFLINE with no pending message - CurrentState currentState1 = new CurrentState(TEST_RESOURCE); - currentState1.setState(PARTITION_0, "OFFLINE"); - currentState1.setSessionId(SESSION_ID); - currentState1.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_1, currentState1); + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + Assert.assertEquals(messages.size(), 1); + + // Should have default currentActiveReplicaNumber (-1) as it's from second top to top state + Message msg = messages.get(0); + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, + "Second top to top state transitions should not have currentActiveReplicaNumber assigned"); + } + + @Test + public void testPendingMessagesDoNotAffectCurrentReplicaCount() throws Exception { + // Test that pending messages don't affect the current active replica count calculation + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Current state: 1 MASTER, 1 OFFLINE + Map currentStateMap = new HashMap<>(); + + CurrentState masterState = new CurrentState(TEST_RESOURCE); + masterState.setState(PARTITION_0, "MASTER"); + masterState.setSessionId(SESSION_ID); + masterState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, masterState); + + CurrentState offlineState1 = new CurrentState(TEST_RESOURCE); + offlineState1.setState(PARTITION_0, "OFFLINE"); + offlineState1.setSessionId(SESSION_ID); + offlineState1.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_1, offlineState1); + + CurrentState offlineState2 = new CurrentState(TEST_RESOURCE); + offlineState2.setState(PARTITION_0, "OFFLINE"); + offlineState2.setSessionId(SESSION_ID); + offlineState2.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_2, offlineState2); CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - // Add pending message to the CurrentStateOutput, not CurrentState - currentStateOutput.setPendingMessage(TEST_RESOURCE, new Partition(PARTITION_0), INSTANCE_0, + // Add pending message for INSTANCE_1: OFFLINE->SLAVE + Message pendingMsg = createMessage("OFFLINE", "SLAVE", INSTANCE_1); + pendingMsg.setMsgId(UUID.randomUUID().toString()); + currentStateOutput.setPendingMessage(TEST_RESOURCE, new Partition(PARTITION_0), INSTANCE_1, pendingMsg); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - // Best possible state wants both to be SLAVE + // Best possible state wants both offline instances to become SLAVE Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "SLAVE"); - partitionMap.put(INSTANCE_1, "SLAVE"); + partitionMap.put(INSTANCE_0, "MASTER"); // No change + partitionMap.put(INSTANCE_1, "SLAVE"); // OFFLINE -> SLAVE (but already has pending message) + partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (new transition) BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); @@ -248,27 +265,140 @@ public void testPrioritizationWithPendingMessages() throws Exception { MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - // Should only generate message for instance 1 (instance 0 already has pending message) + // Should only generate message for INSTANCE_2 (INSTANCE_1 already has pending message) Assert.assertEquals(messages.size(), 1); - Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_1); + Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); - // The current replica number should account for the pending message - // targetActiveReplicaCount = 2 (2 SLAVE), pendingUpwardMessages = 1 - // replicaNumberCounter = 2 - 1 = 1, so new message gets current replica number 1 - Assert.assertTrue(messages.get(0).getCurrentReplicaNumber() >= 0, - "Should assign current replica number considering pending upward messages"); + // The new message should have currentActiveReplicaNumber = current active replica count = 1 (only + // MASTER currently active) + // Note: pending messages should NOT affect the current active replica count calculation + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 1, + "currentActiveReplicaNumber should be based on current active replicas only, not including pending transitions"); } @Test - public void testTransitionFromSecondTopState() throws Exception { - // Test that transitions from second top states are not prioritized + public void testMultiTopStateModel() throws Exception { + // Test multi-top state model (e.g., OFFLINE->ONLINE where ONLINE is the only top state, this + // example does not include ERROR states.) + StateModelDefinition stateModelDefinition = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); + + // Current state: 1 ONLINE, 2 OFFLINE + Map currentStateMap = new HashMap<>(); + + CurrentState onlineState = new CurrentState(TEST_RESOURCE); + onlineState.setState(PARTITION_0, "ONLINE"); + onlineState.setSessionId(SESSION_ID); + onlineState.setStateModelDefRef("OfflineOnline"); + currentStateMap.put(INSTANCE_0, onlineState); + + CurrentState offlineState1 = new CurrentState(TEST_RESOURCE); + offlineState1.setState(PARTITION_0, "OFFLINE"); + offlineState1.setSessionId(SESSION_ID); + offlineState1.setStateModelDefRef("OfflineOnline"); + currentStateMap.put(INSTANCE_1, offlineState1); + + CurrentState offlineState2 = new CurrentState(TEST_RESOURCE); + offlineState2.setState(PARTITION_0, "OFFLINE"); + offlineState2.setSessionId(SESSION_ID); + offlineState2.setStateModelDefRef("OfflineOnline"); + currentStateMap.put(INSTANCE_2, offlineState2); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEventForMultiTop(stateModelDefinition, currentStateOutput); + + // Best possible state wants to add 2 more ONLINE replica + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "ONLINE"); // ONLINE -> ONLINE (no change) + partitionMap.put(INSTANCE_1, "ONLINE"); // OFFLINE -> ONLINE (upward transition) + partitionMap.put(INSTANCE_2, "ONLINE"); // OFFLINE -> ONLINE (upward transition) + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 2 message for OFFLINE->ONLINE transition + Assert.assertEquals(messages.size(), 2); + Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); + Assert.assertEquals(messages.get(1).getTgtName(), INSTANCE_1); + + // For multi-top state model, only top states (ONLINE) and ERROR state count as active + // Current active replicas: 1 ONLINE = 1 + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 1, + "For multi-top state model, currentActiveReplicaNumber should count only top states"); + Assert.assertEquals(messages.get(1).getCurrentActiveReplicaNumber(), 1, + "For multi-top state model, currentActiveReplicaNumber should count only top states"); + } + + @Test + public void testErrorStateIncludedInActiveCount() throws Exception { + // Test that ERROR state replicas are included in active replica count StateModelDefinition stateModelDef = new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - // Set up current state - instance in SLAVE state (second top) + // Current state: 1 MASTER, 1 ERROR, 1 OFFLINE + Map currentStateMap = new HashMap<>(); + + CurrentState masterState = new CurrentState(TEST_RESOURCE); + masterState.setState(PARTITION_0, "MASTER"); + masterState.setSessionId(SESSION_ID); + masterState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, masterState); + + CurrentState errorState = new CurrentState(TEST_RESOURCE); + errorState.setState(PARTITION_0, "ERROR"); + errorState.setSessionId(SESSION_ID); + errorState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_1, errorState); + + CurrentState offlineState = new CurrentState(TEST_RESOURCE); + offlineState.setState(PARTITION_0, "OFFLINE"); + offlineState.setSessionId(SESSION_ID); + offlineState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_2, offlineState); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state wants OFFLINE to become SLAVE + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "MASTER"); // MASTER -> MASTER (no change) + partitionMap.put(INSTANCE_1, "ERROR"); // ERROR -> ERROR (no change) + partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 1 message for OFFLINE->SLAVE transition + Assert.assertEquals(messages.size(), 1); + Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); + + // Current active replicas should include ERROR state: 1 MASTER + 1 ERROR = 2 + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 2, + "currentActiveReplicaNumber should include ERROR state replicas in active count"); + } + + @Test + public void testTransitionFromErrorToOffline() throws Exception { + // Test ERROR→OFFLINE transition (standard recovery path) - should NOT get currentReplicaNumber + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Current state: Instance in ERROR state Map currentStateMap = new HashMap<>(); CurrentState currentState = new CurrentState(TEST_RESOURCE); - currentState.setState(PARTITION_0, "SLAVE"); + currentState.setState(PARTITION_0, "ERROR"); currentState.setSessionId(SESSION_ID); currentState.setStateModelDefRef("MasterSlave"); currentStateMap.put(INSTANCE_0, currentState); @@ -276,9 +406,9 @@ public void testTransitionFromSecondTopState() throws Exception { CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - // Best possible state wants to move to MASTER (top state) + // Best possible state wants to move ERROR to OFFLINE (standard recovery pattern) Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); + partitionMap.put(INSTANCE_0, "OFFLINE"); BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); @@ -291,15 +421,127 @@ public void testTransitionFromSecondTopState() throws Exception { Assert.assertEquals(messages.size(), 1); - // Should have default replica number (-1) as it's from second top state + // Should have default replica number (-1) since it's a downward transition (active to inactive) Message msg = messages.get(0); - Assert.assertEquals(msg.getCurrentReplicaNumber(), -1, - "Transitions from second top states should not be prioritized"); + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, + "ERROR→OFFLINE transitions should not have currentActiveReplicaNumber assigned since it's downward (active to inactive)"); + } + + @Test + public void testDroppedReplicasExcludedFromActiveCount() throws Exception { + // Test that DROPPED replicas are properly excluded from calculations + StateModelDefinition stateModelDef = + new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + + // Current state: 1 MASTER, 1 SLAVE, 1 OFFLINE + Map currentStateMap = new HashMap<>(); + + CurrentState masterState = new CurrentState(TEST_RESOURCE); + masterState.setState(PARTITION_0, "MASTER"); + masterState.setSessionId(SESSION_ID); + masterState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_0, masterState); + + CurrentState slaveState = new CurrentState(TEST_RESOURCE); + slaveState.setState(PARTITION_0, "OFFLINE"); + slaveState.setSessionId(SESSION_ID); + slaveState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_1, slaveState); + + CurrentState offlineState = new CurrentState(TEST_RESOURCE); + offlineState.setState(PARTITION_0, "OFFLINE"); + offlineState.setSessionId(SESSION_ID); + offlineState.setStateModelDefRef("MasterSlave"); + currentStateMap.put(INSTANCE_2, offlineState); + + CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + + // Best possible state: MASTER stays, SLAVE gets DROPPED, OFFLINE becomes SLAVE + Map partitionMap = new HashMap<>(); + partitionMap.put(INSTANCE_0, "MASTER"); // MASTER -> MASTER (no change) + partitionMap.put(INSTANCE_1, "DROPPED"); // OFFLINE -> DROPPED (downward transition) + partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + + process(event); + + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + + // Should generate 2 messages: OFFLINE->DROPPED and OFFLINE->SLAVE + Assert.assertEquals(messages.size(), 2); + + Message upwardTransitionMsg = null; + Message droppedTransitionMsg = null; + for (Message msg : messages) { + if (msg.getTgtName().equals(INSTANCE_2) && msg.getToState().equals("SLAVE")) { + upwardTransitionMsg = msg; + } else if (msg.getTgtName().equals(INSTANCE_1) && msg.getToState().equals("DROPPED")) { + droppedTransitionMsg = msg; + } + } + + Assert.assertNotNull(upwardTransitionMsg, "Should have upward state transition message"); + Assert.assertNotNull(droppedTransitionMsg, "Should have dropped state transition message"); + + // Upward transition should get currentActiveReplicaNumber = current active replicas + // Current active replicas: 1 (MASTER = 1) + Assert.assertEquals(upwardTransitionMsg.getCurrentActiveReplicaNumber(), 1, + "Upward transition should have currentActiveReplicaNumber = current active replica count"); + + // DROPPED transition should not get currentActiveReplicaNumber (downward transition) + Assert.assertEquals(droppedTransitionMsg.getCurrentActiveReplicaNumber(), -1, + "DROPPED transition should not have currentActiveReplicaNumber assigned"); + } + + private ClusterEvent prepareClusterEventForMultiTop(StateModelDefinition stateModelDefinition, + CurrentStateOutput currentStateOutput) { + ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); + + event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput); + event.addAttribute(AttributeName.CURRENT_STATE_EXCLUDING_UNKNOWN.name(), currentStateOutput); + + // Mock HelixManager + HelixManager manager = mock(HelixManager.class); + when(manager.getInstanceName()).thenReturn("Controller"); + when(manager.getSessionId()).thenReturn(SESSION_ID); + + // Mock HelixDataAccessor + HelixDataAccessor dataAccessor = mock(HelixDataAccessor.class); + when(manager.getHelixDataAccessor()).thenReturn(dataAccessor); + + event.addAttribute(AttributeName.helixmanager.name(), manager); + + // Setup ResourceControllerDataProvider + ResourceControllerDataProvider cache = mock(ResourceControllerDataProvider.class); + when(cache.getClusterConfig()).thenReturn(new ClusterConfig("TestCluster")); + when(cache.getStateModelDef("OfflineOnline")).thenReturn(stateModelDefinition); + + // Mock live instances + Map liveInstances = new HashMap<>(); + liveInstances.put(INSTANCE_0, createLiveInstance(INSTANCE_0)); + liveInstances.put(INSTANCE_1, createLiveInstance(INSTANCE_1)); + liveInstances.put(INSTANCE_2, createLiveInstance(INSTANCE_2)); + when(cache.getLiveInstances()).thenReturn(liveInstances); + + event.addAttribute(AttributeName.ControllerDataProvider.name(), cache); + + // Setup resources + Map resourceMap = new HashMap<>(); + Resource resource = new Resource(TEST_RESOURCE); + resource.setStateModelDefRef("OfflineOnline"); + resource.addPartition(PARTITION_0); + resourceMap.put(TEST_RESOURCE, resource); + event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), resourceMap); + return event; } @Test public void testReplicaCountCalculationWithDroppedReplicas() throws Exception { - // Test that DROPPED replicas are excluded from target active replica count StateModelDefinition stateModelDef = new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); @@ -332,9 +574,9 @@ public void testReplicaCountCalculationWithDroppedReplicas() throws Exception { // Best possible state: MASTER stays, one SLAVE gets DROPPED, one OFFLINE becomes SLAVE Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); // No change + partitionMap.put(INSTANCE_0, "MASTER"); // No change partitionMap.put(INSTANCE_1, "DROPPED"); // SLAVE -> DROPPED (excluded from active count) - partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); @@ -361,15 +603,13 @@ public void testReplicaCountCalculationWithDroppedReplicas() throws Exception { Assert.assertNotNull(upwardTransitionMsg, "Should have upward state transition message"); Assert.assertNotNull(droppedTransitionMsg, "Should have dropped state transition message"); - // Upward transition should get replica number - // targetActiveReplicaCount = 2 (1 MASTER + 1 SLAVE, DROPPED excluded) - // replicaNumberCounter = 2 - 0 = 2, upward transition gets current replica number 2 - Assert.assertEquals(upwardTransitionMsg.getCurrentReplicaNumber(), 2, - "Upward state transition message should have current replica number assigned based on active replica count excluding DROPPED"); + // Upward transition should get currentActiveReplicaNumber + Assert.assertEquals(upwardTransitionMsg.getCurrentActiveReplicaNumber(), 2, + "Upward state transition message should have current active replica number assigned based on active replica count excluding DROPPED"); - // DROPPED transition should not get current replica number (it's a downward transition anyway) - Assert.assertEquals(droppedTransitionMsg.getCurrentReplicaNumber(), -1, - "DROPPED state transition message should not have current replica number assigned."); + // DROPPED transition should not get current active replica number (it's a downward transition anyway) + Assert.assertEquals(droppedTransitionMsg.getCurrentActiveReplicaNumber(), -1, + "DROPPED state transition message should not have current active replica number assigned."); } private ClusterEvent prepareClusterEvent(StateModelDefinition stateModelDef, From 0f84ac97ae7dd68845dec82fef54cc1ca54bff56 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Mon, 16 Jun 2025 15:52:24 -0700 Subject: [PATCH 05/11] Distinguish between SingleTop and MultiTop, Add relevant tests and reformat tests. --- .../stages/MessageGenerationPhase.java | 159 +++- .../apache/helix/model/OnlineOfflineSMD.java | 21 +- .../TestPrioritizationMessageGeneration.java | 837 +++++++----------- 3 files changed, 453 insertions(+), 564 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 81505b83b4..0fe8233833 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -26,6 +26,7 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; import org.apache.helix.HelixDataAccessor; import org.apache.helix.HelixDefinedState; @@ -44,6 +45,7 @@ import org.apache.helix.model.IdealState; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; +import org.apache.helix.model.OnlineOfflineSMD; import org.apache.helix.model.Partition; import org.apache.helix.model.Resource; import org.apache.helix.model.ResourceConfig; @@ -57,17 +59,17 @@ * Compares the currentState, pendingState with IdealState and generate messages */ public class MessageGenerationPhase extends AbstractBaseStage { - private final static String NO_DESIRED_STATE = "NoDesiredState"; + private static final String NO_DESIRED_STATE = "NoDesiredState"; // If we see there is any invalid pending message leaving on host, i.e. message // tells participant to change from SLAVE to MASTER, and the participant is already // at MASTER state, we wait for timeout and if the message is still not cleaned up by // participant, controller will cleanup them proactively to unblock further state // transition - public final static long DEFAULT_OBSELETE_MSG_PURGE_DELAY = HelixUtil + public static final long DEFAULT_OBSELETE_MSG_PURGE_DELAY = HelixUtil .getSystemPropertyAsLong(SystemPropertyKeys.CONTROLLER_MESSAGE_PURGE_DELAY, 60 * 1000); - private final static String PENDING_MESSAGE = "pending message"; - private final static String STALE_MESSAGE = "stale message"; + private static final String PENDING_MESSAGE = "pending message"; + private static final String STALE_MESSAGE = "stale message"; private static Logger logger = LoggerFactory.getLogger(MessageGenerationPhase.class); @@ -163,31 +165,15 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr // desired-state->list of generated-messages Map> messageMap = new HashMap<>(); - /** - * Calculate the current active replica count based on state model type. - * This represents the number of replicas currently serving traffic for this partition - * Active replicas include: top states, secondary top states (for single-top state models), - * and ERROR states. - * All qualifying state transitions for this partition will receive this same value, - * allowing clients to understand the current availability level and prioritize accordingly. + /* + Calculate the current active replica count based on state model type. + This represents the number of replicas currently serving traffic for this partition + Active replicas include: top states, secondary top states (for single-top state models if + they exist)and ERROR states. + All qualifying state transitions for this partition will receive this same value, + allowing clients to understand the current availability level and prioritize accordingly. */ - int currentActiveReplicaCount; - if (stateModelDef.isSingleTopStateModel()) { - // For single-top state models (e.g., OFFLINE→STANDBY→ONLINE) - // Count replicas in top state, secondary top state and ERROR state - currentActiveReplicaCount = (int) currentStateMap.values().stream() - .filter(state -> stateModelDef.getTopState().contains(state) - || stateModelDef.getSecondTopStates().contains(state) - || HelixDefinedState.ERROR.name().equals(state)) - .count(); - } else { - // For multi-top state models (e.g., OFFLINE→ONLINE) - // Count replicas that will be in top state and ERROR state - currentActiveReplicaCount = (int) currentStateMap.values().stream() - .filter(state -> stateModelDef.getTopState().contains(state) - || HelixDefinedState.ERROR.name().equals(state)) - .count(); - } + int currentActiveReplicaCount = calculateCurrentActiveReplicaCount(currentStateMap, stateModelDef); for (String instanceName : instanceStateMap.keySet()) { @@ -284,12 +270,12 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr int currentActiveReplicaNumber = -1; // -1 indicates no prioritization metadata, for eg: // Downward ST messages get a -1. - /** - * Assign currentActiveReplicaNumber for qualifying upward state transitions. - * Criteria for assignment: - * - Must be an upward state transition according to state model - * - Current state must not be considered active (according to state model type) - * - Target state must be considered active (according to state model type) + /* + Assign currentActiveReplicaNumber for qualifying upward state transitions. + Criteria for assignment: + - Must be an upward state transition according to state model + - Current state must not be considered active (according to state model type) + - Target state must be considered active (according to state model type) */ if (stateModelDef.isUpwardStateTransition(currentState, nextState) && !isCurrentlyActive(currentState, stateModelDef, @@ -341,9 +327,106 @@ && isTargetActive(nextState, stateModelDef, } // end of for-each-partition } + /** + * Calculate the current active replica count based on state model type. + * This method determines how many replicas are currently serving traffic for a partition by + * analyzing the current state distribution and applying state model-specific rules. The count includes + * replicas in top states, secondary top states (where applicable), and ERROR states since helix considers + * them active. + * State model handling: + * - Single-top state models: Differentiates between patterns with and without secondary top + * states + * - ONLINE-OFFLINE: Counts ONLINE + ERROR states only + * - MASTER-SLAVE-OFFLINE: Counts MASTER + SLAVE + ERROR states + * - ONLINE-STANDBY-OFFLINE: Counts ONLINE + STANDBY + ERROR states + * - Multi-top state models: Counts only top states + ERROR states + * @param currentStateMap Map of instance name to current state for this partition, representing + * the actual state of each replica before any pending transitions + * @param stateModelDef State model definition containing the state hierarchy and transition rules + * used to determine which states are considered active + * @return The number of replicas currently in active states, used to determine the + * currentActiveReplicaNumber for the partition. + */ + private int calculateCurrentActiveReplicaCount(Map currentStateMap, + StateModelDefinition stateModelDef) { + if (stateModelDef.isSingleTopStateModel()) { + return calculateSingleTopStateActiveCount(currentStateMap, stateModelDef); + } else { + return calculateMultiTopStateActiveCount(currentStateMap, stateModelDef); + } + } + + /** + * Calculate active replica count for single-top state models. + * Single-top state models have different active state definitions: + * - ONLINE-OFFLINE: Only ONLINE (top state) + ERROR are active + * - ONLINE-STANDBY-OFFLINE: ONLINE (top state) + STANDBY (secondary top) + ERROR are active + * Note: We need to identify true secondary states (like STANDBY) vs transition-only + * states (like OFFLINE). + */ + private int calculateSingleTopStateActiveCount(Map currentStateMap, + StateModelDefinition stateModelDef) { + List trueSecondaryTopStates = getTrueSecondaryTopStates(stateModelDef); + if (trueSecondaryTopStates.isEmpty()) { + // No true secondary states exist (e.g., ONLINE-OFFLINE pattern) + // Count: top + ERROR states only + // Example: OnlineOffline has getSecondTopStates()=["OFFLINE"] but OFFLINE is non-serving state + // so trueSecondaryTopStates=[] and we only count ONLINE + ERROR + return (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) + || HelixDefinedState.ERROR.name().equals(state)) + .count(); + } else { + // True secondary states exist (e.g., MASTER-SLAVE, ONLINE-STANDBY, LEADER-STANDBY, OnlineOfflineWithBootstrap) + // Count: top + true secondary top + ERROR states + // Example for MasterSlave: trueSecondaryTopStates=["SLAVE"] + // Example for OnlineOfflineWithBootstrap: trueSecondaryTopStates=["BOOTSTRAP"] + return (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) + || trueSecondaryTopStates.contains(state) + || HelixDefinedState.ERROR.name().equals(state)) + .count(); + } + } + + /** + * Get true secondary top states - states that: + * 1. Are not the top state itself (avoid double-counting) + * 2. Are not non-serving states like OFFLINE and DROPPED. + * Reasons for elimination: + * - getSecondTopStates() can include the top state itself in some state models. + * Example - OnlineOfflineWithBootstrap: + * topState="ONLINE", getSecondTopStates()=["ONLINE", "BOOTSTRAP"] + * After filtering: trueSecondaryTopStates=["BOOTSTRAP"] (removes "ONLINE" as it is top state.) + * - getSecondTopStates() can also include OFFLINE as a secondary top state in some state models. + * Example - OnlineOffline: + * getSecondTopStates() = ["OFFLINE"] as it transitions to ONLINE. + * After filtering: trueSecondaryTopStates=[] (removes "OFFLINE" as it's not a serving state). + * @param stateModelDef State model definition containing state hierarchy information + */ + private List getTrueSecondaryTopStates(StateModelDefinition stateModelDef) { + return stateModelDef.getSecondTopStates().stream() + .filter(state -> !stateModelDef.getTopState().equals(state)) // Remove top state duplicates + .filter(state -> !OnlineOfflineSMD.States.OFFLINE.name().equals(state) + && !HelixDefinedState.DROPPED.name().equals(state)) // Remove non-serving states + .collect(Collectors.toList()); + } + + /** + * Calculate active replica count for multi-top state models. + * For multi-top state models (e.g., OFFLINE→ONLINE), count only top states + ERROR. + */ + private int calculateMultiTopStateActiveCount(Map currentStateMap, + StateModelDefinition stateModelDef) { + return (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) + || HelixDefinedState.ERROR.name().equals(state)) + .count(); + } + /** * Determines if the given current state is considered active based on the state model type. - * For single-top state models, top, secondary top, and ERROR states are active. + * For single-top state models, top, true secondary top, and ERROR states are active. * For multi-top state models, top and ERROR states are active. * ERROR state replicas are considered active in HELIX as they do not affect availability. * @param currentState The current state to check @@ -359,7 +442,7 @@ private boolean isCurrentlyActive(String currentState, StateModelDefinition stat } if (isSingleTopState) { return stateModelDef.getTopState().contains(currentState) - || stateModelDef.getSecondTopStates().contains(currentState); + || getTrueSecondaryTopStates(stateModelDef).contains(currentState); } else { return stateModelDef.getTopState().contains(currentState); } @@ -367,7 +450,7 @@ private boolean isCurrentlyActive(String currentState, StateModelDefinition stat /** * Determines if the given target state is considered active based on the state model type. - * For single-top state models, both top,secondary top and ERROR states are active. + * For single-top state models, both top,true secondary top and ERROR states are active. * For multi-top state models, top and ERROR states are active. * @param targetState The target state to check * @param stateModelDef State model definition containing state hierarchy information @@ -382,7 +465,7 @@ private boolean isTargetActive(String targetState, StateModelDefinition stateMod } if (isSingleTopState) { return stateModelDef.getTopState().contains(targetState) - || stateModelDef.getSecondTopStates().contains(targetState); + || getTrueSecondaryTopStates(stateModelDef).contains(targetState); } else { return stateModelDef.getTopState().contains(targetState); } diff --git a/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java b/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java index fd97c7ba9f..ab207218ac 100644 --- a/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java +++ b/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java @@ -42,11 +42,21 @@ public OnlineOfflineSMD() { } /** - * Build OnlineOffline state model definition - * @return + * Build OnlineOffline state model definition with default replica count + * @return StateModelDefinition for OnlineOffline model */ public static StateModelDefinition build() { - StateModelDefinition.Builder builder =new StateModelDefinition.Builder(name); + return build(StateModelDefinition.STATE_REPLICA_COUNT_ALL_REPLICAS); + } + + /** + * Build OnlineOffline state model definition with custom replica count + * @param instanceCount the maximum number of instances that can be in ONLINE state + * @return StateModelDefinition for OnlineOffline model + */ + public static StateModelDefinition build(String instanceCount) { + StateModelDefinition.Builder builder = new StateModelDefinition.Builder(name); + // init state builder.initialState(States.OFFLINE.name()); @@ -62,9 +72,8 @@ public static StateModelDefinition build() { builder.addTransition(States.OFFLINE.name(), States.ONLINE.name(), 1); builder.addTransition(States.OFFLINE.name(), HelixDefinedState.DROPPED.name()); - // bounds - builder.dynamicUpperBound(States.ONLINE.name(), - StateModelDefinition.STATE_REPLICA_COUNT_ALL_REPLICAS); + // bounds - uses the instanceCount parameter instead of constant + builder.dynamicUpperBound(States.ONLINE.name(), instanceCount); return builder.build(); } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index fc81d1545b..1e6369e15e 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -31,14 +31,16 @@ import org.apache.helix.model.CurrentState; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; +import org.apache.helix.model.OnlineOfflineSMD; import org.apache.helix.model.Partition; import org.apache.helix.model.Resource; import org.apache.helix.model.StateModelDefinition; -import org.apache.helix.tools.StateModelConfigGenerator; import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.testng.Assert; import org.testng.annotations.Test; +import static org.apache.helix.tools.StateModelConfigGenerator.generateConfigForMasterSlave; +import static org.apache.helix.tools.StateModelConfigGenerator.generateConfigForOnlineOffline; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -51,677 +53,472 @@ public class TestPrioritizationMessageGeneration extends MessageGenerationPhase private static final String INSTANCE_1 = "localhost_1"; private static final String INSTANCE_2 = "localhost_2"; private static final String INSTANCE_3 = "localhost_3"; + private static final String INSTANCE_4 = "localhost_4"; private static final String SESSION_ID = "123"; + // === Tests for upward transitions and replica counting === + @Test public void testCurrentReplicaCountForUpwardTransitions() throws Exception { - - // Test prioritization for upward state transitions from non-second top states - // to second top or top states - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - - // Set up current states - Map currentStateMap = setupCurrentStatesForPrioritizationCase1(); - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - - // Best possible state wants to move some nodes to SLAVE - Map partitionMap = new HashMap<>(); - // No ST messages should be generated as current state is same as desired state - partitionMap.put(INSTANCE_0, "MASTER"); // Master -> Master - partitionMap.put(INSTANCE_1, "SLAVE"); // Slave -> Slave - - // upward to second top - partitionMap.put(INSTANCE_2, "SLAVE"); // Offline -> Slave - partitionMap.put(INSTANCE_3, "SLAVE"); // Offline -> Slave - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); - - process(event); - - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - - // Should generate 2 messages - Assert.assertEquals(messages.size(), 2); - - // Both messages should have the same currentActiveReplicaNumber = current active replica count + // Test: Upward transitions from non-second top states should receive currentActiveReplicaNumber + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); + // Setup: 1 MASTER, 1 SLAVE, 2 OFFLINE (current active = 2) + Map currentStates = createCurrentStates(Map.of(INSTANCE_0, "MASTER", + INSTANCE_1, "SLAVE", INSTANCE_2, "OFFLINE", INSTANCE_3, "OFFLINE"), "MasterSlave"); + // Action: Move 2 OFFLINE instances to SLAVE (upward transitions) + Map bestPossible = Map.of(INSTANCE_0, "MASTER", // No change + INSTANCE_1, "SLAVE", // No change + INSTANCE_2, "SLAVE", // OFFLINE -> SLAVE (upward) + INSTANCE_3, "SLAVE" // OFFLINE -> SLAVE (upward) + ); + + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 4); + + // Verify: 2 messages generated, both with current active replica count = 2 // Current active replicas: 1 MASTER + 1 SLAVE = 2 + Assert.assertEquals(messages.size(), 2); for (Message msg : messages) { Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 2, - "All upward transitions should have currentActiveReplicaNumber = current active replica count (2)"); - Assert.assertTrue(msg.getTgtName().equals(INSTANCE_2) || msg.getTgtName().equals(INSTANCE_3), - "Messages should be for instances transitioning from OFFLINE to SLAVE"); + "Upward transitions should have currentActiveReplicaNumber = current active replica count"); + Assert.assertTrue(msg.getTgtName().equals(INSTANCE_2) || msg.getTgtName().equals(INSTANCE_3)); } } @Test public void testZeroReplicaScenario() throws Exception { - // Test scenario with 0 current active replicas (all OFFLINE) - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // Test: All instances starting from OFFLINE (0 active replicas) + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - // Current state: All instances in OFFLINE state - Map currentStateMap = setupCurrentStatesForPrioritizationCase2(); - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + // Setup: All instances OFFLINE (current active = 0) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "OFFLINE", INSTANCE_1, "OFFLINE", INSTANCE_2, "OFFLINE"), "MasterSlave"); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + // Action: Create 1 MASTER, 2 SLAVE from all OFFLINE + Map bestPossible = Map.of(INSTANCE_0, "SLAVE", // OFFLINE -> SLAVE (upward) + INSTANCE_1, "MASTER", // OFFLINE -> MASTER (upward) + INSTANCE_2, "SLAVE" // OFFLINE -> SLAVE (upward) + ); - // Best possible state wants to create 1 MASTER, 2 SLAVE - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "SLAVE"); // Offline -> Slave (upward to second top) - partitionMap.put(INSTANCE_1, "MASTER"); // Offline -> Master (upward to top) - partitionMap.put(INSTANCE_2, "SLAVE"); // Offline -> Slave (upward to second top) - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 3); - process(event); - - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - - // Should generate 3 messages + // Verify: All messages have currentActiveReplicaNumber = 0 Assert.assertEquals(messages.size(), 3); - - // All messages should have currentActiveReplicaNumber = 0 (current active replica count) for (Message msg : messages) { Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 0, "All upward transitions should have currentActiveReplicaNumber = 0"); } } + // === Tests for non-upward transitions === + @Test public void testNoReplicaNumberForNonUpwardTransitions() throws Exception { - // Test that non-upward transitions don't get currentActiveReplicaNumber assigned - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // Test: Downward transitions should not receive currentActiveReplicaNumber + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - // Current state: Instance in SLAVE state - Map currentStateMap = new HashMap<>(); - CurrentState currentState = new CurrentState(TEST_RESOURCE); - currentState.setState(PARTITION_0, "SLAVE"); - currentState.setSessionId(SESSION_ID); - currentState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, currentState); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + Map currentStates = + createCurrentStates(Map.of(INSTANCE_0, "SLAVE"), "MasterSlave"); - // Best possible state wants to move to OFFLINE (downward transition) - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "OFFLINE"); - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + // Action: SLAVE -> OFFLINE (downward transition) + Map bestPossible = Map.of(INSTANCE_0, "OFFLINE"); - process(event); - - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 1); + // Verify: Downward transition gets default value (-1) Assert.assertEquals(messages.size(), 1); - - // Should have default currentActiveReplicaNumber (-1) for non-upward transition - Message msg = messages.get(0); - Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), -1, "Non-upward state transitions should not have currentActiveReplicaNumber assigned"); } @Test public void testNoReplicaNumberForSecondTopToTopTransitions() throws Exception { - // Tests No currentActiveReplicaNumber assigned for Second Top -> Top State Transitions for Single-Top-State - // State Model - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // Test: SLAVE -> MASTER (second top to top) should not receive currentActiveReplicaNumber + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - // Current state: Instance in SLAVE state (second top) - Map currentStateMap = new HashMap<>(); - CurrentState currentState = new CurrentState(TEST_RESOURCE); - currentState.setState(PARTITION_0, "SLAVE"); - currentState.setSessionId(SESSION_ID); - currentState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, currentState); + Map currentStates = + createCurrentStates(Map.of(INSTANCE_0, "SLAVE"), "MasterSlave"); - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + // Action: SLAVE -> MASTER (second top to top transition) + Map bestPossible = Map.of(INSTANCE_0, "MASTER"); - // Best possible state wants to promote to MASTER (second top -> top transition) - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); - - process(event); - - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 1); + // Verify: Second top to top transition gets default value (-1) Assert.assertEquals(messages.size(), 1); - - // Should have default currentActiveReplicaNumber (-1) as it's from second top to top state - Message msg = messages.get(0); - Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), -1, "Second top to top state transitions should not have currentActiveReplicaNumber assigned"); } + // === Tests for pending messages === + @Test public void testPendingMessagesDoNotAffectCurrentReplicaCount() throws Exception { - // Test that pending messages don't affect the current active replica count calculation - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - - // Current state: 1 MASTER, 1 OFFLINE - Map currentStateMap = new HashMap<>(); - - CurrentState masterState = new CurrentState(TEST_RESOURCE); - masterState.setState(PARTITION_0, "MASTER"); - masterState.setSessionId(SESSION_ID); - masterState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, masterState); + // Test: SLAVE -> MASTER (second top to top) should not receive currentActiveReplicaNumber + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - CurrentState offlineState1 = new CurrentState(TEST_RESOURCE); - offlineState1.setState(PARTITION_0, "OFFLINE"); - offlineState1.setSessionId(SESSION_ID); - offlineState1.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_1, offlineState1); + // Setup: 1 MASTER, 2 OFFLINE (current active = 1) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "OFFLINE", INSTANCE_2, "OFFLINE"), "MasterSlave"); - CurrentState offlineState2 = new CurrentState(TEST_RESOURCE); - offlineState2.setState(PARTITION_0, "OFFLINE"); - offlineState2.setSessionId(SESSION_ID); - offlineState2.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_2, offlineState2); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); + CurrentStateOutput currentStateOutput = createCurrentStateOutput(currentStates); // Add pending message for INSTANCE_1: OFFLINE->SLAVE Message pendingMsg = createMessage("OFFLINE", "SLAVE", INSTANCE_1); - pendingMsg.setMsgId(UUID.randomUUID().toString()); currentStateOutput.setPendingMessage(TEST_RESOURCE, new Partition(PARTITION_0), INSTANCE_1, pendingMsg); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput, 3); - // Best possible state wants both offline instances to become SLAVE - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); // No change - partitionMap.put(INSTANCE_1, "SLAVE"); // OFFLINE -> SLAVE (but already has pending message) - partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (new transition) - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + // Action: Both offline instances should become SLAVE + Map bestPossible = Map.of(INSTANCE_0, "MASTER", // No change + INSTANCE_1, "SLAVE", // Has pending message, no new message + INSTANCE_2, "SLAVE" // OFFLINE -> SLAVE (new transition) + ); + setBestPossibleState(event, bestPossible); process(event); MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - // Should only generate message for INSTANCE_2 (INSTANCE_1 already has pending message) + // Verify: Only new message for INSTANCE_2, with current active count = 1 (ignoring pending) Assert.assertEquals(messages.size(), 1); Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); - - // The new message should have currentActiveReplicaNumber = current active replica count = 1 (only - // MASTER currently active) - // Note: pending messages should NOT affect the current active replica count calculation Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 1, "currentActiveReplicaNumber should be based on current active replicas only, not including pending transitions"); } + // === Tests for different state model types === + @Test - public void testMultiTopStateModel() throws Exception { - // Test multi-top state model (e.g., OFFLINE->ONLINE where ONLINE is the only top state, this - // example does not include ERROR states.) - StateModelDefinition stateModelDefinition = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline()); + public void testSingleTopStateModelWithoutSecondaryTop() throws Exception { + // Test: ONLINE-OFFLINE model (single top without secondary) - only top + ERROR count as active + StateModelDefinition onlineOfflineStateModel = OnlineOfflineSMD.build("1"); - // Current state: 1 ONLINE, 2 OFFLINE - Map currentStateMap = new HashMap<>(); + // Verify this is a single-top state model + Assert.assertTrue(onlineOfflineStateModel.isSingleTopStateModel(), + "ONLINE-OFFLINE should be a single-top state model"); - CurrentState onlineState = new CurrentState(TEST_RESOURCE); - onlineState.setState(PARTITION_0, "ONLINE"); - onlineState.setSessionId(SESSION_ID); - onlineState.setStateModelDefRef("OfflineOnline"); - currentStateMap.put(INSTANCE_0, onlineState); - - CurrentState offlineState1 = new CurrentState(TEST_RESOURCE); - offlineState1.setState(PARTITION_0, "OFFLINE"); - offlineState1.setSessionId(SESSION_ID); - offlineState1.setStateModelDefRef("OfflineOnline"); - currentStateMap.put(INSTANCE_1, offlineState1); - - CurrentState offlineState2 = new CurrentState(TEST_RESOURCE); - offlineState2.setState(PARTITION_0, "OFFLINE"); - offlineState2.setSessionId(SESSION_ID); - offlineState2.setStateModelDefRef("OfflineOnline"); - currentStateMap.put(INSTANCE_2, offlineState2); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEventForMultiTop(stateModelDefinition, currentStateOutput); - - // Best possible state wants to add 2 more ONLINE replica - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "ONLINE"); // ONLINE -> ONLINE (no change) - partitionMap.put(INSTANCE_1, "ONLINE"); // OFFLINE -> ONLINE (upward transition) - partitionMap.put(INSTANCE_2, "ONLINE"); // OFFLINE -> ONLINE (upward transition) + // Setup: 0 ONLINE, 1 ERROR, 1 OFFLINE (current active = 1: ERROR only) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "OFFLINE", INSTANCE_1, "ERROR", INSTANCE_2, "OFFLINE"), "OnlineOffline"); - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + // Action: One OFFLINE becomes ONLINE + Map bestPossible = Map.of(INSTANCE_0, "ONLINE", // OFFLINE -> ONLINE (upward) + INSTANCE_1, "ERROR", // No change + INSTANCE_2, "OFFLINE" // No change + ); - process(event); + List messages = processAndGetMessagesForOnlineOffline(onlineOfflineStateModel, + currentStates, bestPossible, 3); - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - - // Should generate 2 message for OFFLINE->ONLINE transition - Assert.assertEquals(messages.size(), 2); - Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); - Assert.assertEquals(messages.get(1).getTgtName(), INSTANCE_1); - - // For multi-top state model, only top states (ONLINE) and ERROR state count as active - // Current active replicas: 1 ONLINE = 1 + // Verify: Current active = 1 (0 ONLINE + 1 ERROR) + Assert.assertEquals(messages.size(), 1); Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 1, - "For multi-top state model, currentActiveReplicaNumber should count only top states"); - Assert.assertEquals(messages.get(1).getCurrentActiveReplicaNumber(), 1, - "For multi-top state model, currentActiveReplicaNumber should count only top states"); + "Single-top without secondary: only top states + ERROR count as active"); } @Test - public void testErrorStateIncludedInActiveCount() throws Exception { - // Test that ERROR state replicas are included in active replica count - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + public void testSingleTopStateModelWithSecondaryTop() throws Exception { + // Test: MASTER-SLAVE model (single top with secondary) - top + secondary + ERROR count as + // active + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - // Current state: 1 MASTER, 1 ERROR, 1 OFFLINE - Map currentStateMap = new HashMap<>(); - - CurrentState masterState = new CurrentState(TEST_RESOURCE); - masterState.setState(PARTITION_0, "MASTER"); - masterState.setSessionId(SESSION_ID); - masterState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, masterState); - - CurrentState errorState = new CurrentState(TEST_RESOURCE); - errorState.setState(PARTITION_0, "ERROR"); - errorState.setSessionId(SESSION_ID); - errorState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_1, errorState); - - CurrentState offlineState = new CurrentState(TEST_RESOURCE); - offlineState.setState(PARTITION_0, "OFFLINE"); - offlineState.setSessionId(SESSION_ID); - offlineState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_2, offlineState); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - - // Best possible state wants OFFLINE to become SLAVE - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); // MASTER -> MASTER (no change) - partitionMap.put(INSTANCE_1, "ERROR"); // ERROR -> ERROR (no change) - partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + // Setup: 1 MASTER, 2 SLAVE, 1 ERROR, 1 OFFLINE (current active = 4) + Map currentStates = + createCurrentStates(Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "SLAVE", INSTANCE_2, "SLAVE", + INSTANCE_3, "ERROR", INSTANCE_4, "OFFLINE"), "MasterSlave"); - process(event); + // Action: OFFLINE becomes SLAVE + Map bestPossible = Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "SLAVE", INSTANCE_2, + "SLAVE", INSTANCE_3, "ERROR", INSTANCE_4, "SLAVE" // OFFLINE -> SLAVE (upward) + ); - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 5); - // Should generate 1 message for OFFLINE->SLAVE transition + // Verify: Current active = 4 (1 MASTER + 2 SLAVE + 1 ERROR) Assert.assertEquals(messages.size(), 1); - Assert.assertEquals(messages.get(0).getTgtName(), INSTANCE_2); - - // Current active replicas should include ERROR state: 1 MASTER + 1 ERROR = 2 - Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 2, - "currentActiveReplicaNumber should include ERROR state replicas in active count"); + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 4, + "Single-top with secondary: top + secondary top + ERROR count as active"); } @Test - public void testTransitionFromErrorToOffline() throws Exception { - // Test ERROR→OFFLINE transition (standard recovery path) - should NOT get currentReplicaNumber - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); - - // Current state: Instance in ERROR state - Map currentStateMap = new HashMap<>(); - CurrentState currentState = new CurrentState(TEST_RESOURCE); - currentState.setState(PARTITION_0, "ERROR"); - currentState.setSessionId(SESSION_ID); - currentState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, currentState); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - - // Best possible state wants to move ERROR to OFFLINE (standard recovery pattern) - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "OFFLINE"); - - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + public void testMultiTopStateModel() throws Exception { + // Test: Multi-top state model - only top states + ERROR count as active + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForOnlineOffline()); - process(event); + // Setup: 1 ONLINE, 2 OFFLINE (current active = 1) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "ONLINE", INSTANCE_1, "OFFLINE", INSTANCE_2, "OFFLINE"), + "OfflineOnline"); - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + // Action: Both OFFLINE become ONLINE + Map bestPossible = Map.of(INSTANCE_0, "ONLINE", // No change + INSTANCE_1, "ONLINE", // OFFLINE -> ONLINE (upward) + INSTANCE_2, "ONLINE" // OFFLINE -> ONLINE (upward) + ); - Assert.assertEquals(messages.size(), 1); + List messages = + processAndGetMessagesForOnlineOffline(stateModelDef, currentStates, bestPossible, 3); - // Should have default replica number (-1) since it's a downward transition (active to inactive) - Message msg = messages.get(0); - Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), -1, - "ERROR→OFFLINE transitions should not have currentActiveReplicaNumber assigned since it's downward (active to inactive)"); + // Verify: Current active = 1 (only ONLINE states count) + Assert.assertEquals(messages.size(), 2); + for (Message msg : messages) { + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 1, + "Multi-top state model: only top states count as active"); + } } - @Test - public void testDroppedReplicasExcludedFromActiveCount() throws Exception { - // Test that DROPPED replicas are properly excluded from calculations - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // === Tests for ERROR state handling === - // Current state: 1 MASTER, 1 SLAVE, 1 OFFLINE - Map currentStateMap = new HashMap<>(); + @Test + public void testErrorStateIncludedInActiveCount() throws Exception { + // Test: ERROR states are always counted as active + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - CurrentState masterState = new CurrentState(TEST_RESOURCE); - masterState.setState(PARTITION_0, "MASTER"); - masterState.setSessionId(SESSION_ID); - masterState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, masterState); - - CurrentState slaveState = new CurrentState(TEST_RESOURCE); - slaveState.setState(PARTITION_0, "OFFLINE"); - slaveState.setSessionId(SESSION_ID); - slaveState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_1, slaveState); - - CurrentState offlineState = new CurrentState(TEST_RESOURCE); - offlineState.setState(PARTITION_0, "OFFLINE"); - offlineState.setSessionId(SESSION_ID); - offlineState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_2, offlineState); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - - // Best possible state: MASTER stays, SLAVE gets DROPPED, OFFLINE becomes SLAVE - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); // MASTER -> MASTER (no change) - partitionMap.put(INSTANCE_1, "DROPPED"); // OFFLINE -> DROPPED (downward transition) - partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + // Setup: 1 MASTER, 1 ERROR, 1 OFFLINE (current active = 2) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "ERROR", INSTANCE_2, "OFFLINE"), "MasterSlave"); - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + // Action: OFFLINE becomes SLAVE + Map bestPossible = + Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "ERROR", INSTANCE_2, "SLAVE"); - process(event); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 3); - MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + // Verify: Current active = 2 (1 MASTER + 1 ERROR) + Assert.assertEquals(messages.size(), 1); + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), 2, + "ERROR state replicas should be included in active count"); + } - // Should generate 2 messages: OFFLINE->DROPPED and OFFLINE->SLAVE - Assert.assertEquals(messages.size(), 2); + @Test + public void testTransitionFromErrorToOffline() throws Exception { + // Test: ERROR -> OFFLINE is a downward transition (active to inactive) + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - Message upwardTransitionMsg = null; - Message droppedTransitionMsg = null; - for (Message msg : messages) { - if (msg.getTgtName().equals(INSTANCE_2) && msg.getToState().equals("SLAVE")) { - upwardTransitionMsg = msg; - } else if (msg.getTgtName().equals(INSTANCE_1) && msg.getToState().equals("DROPPED")) { - droppedTransitionMsg = msg; - } - } + Map currentStates = + createCurrentStates(Map.of(INSTANCE_0, "ERROR"), "MasterSlave"); - Assert.assertNotNull(upwardTransitionMsg, "Should have upward state transition message"); - Assert.assertNotNull(droppedTransitionMsg, "Should have dropped state transition message"); + // Action: ERROR -> OFFLINE (standard recovery pattern) + Map bestPossible = Map.of(INSTANCE_0, "OFFLINE"); - // Upward transition should get currentActiveReplicaNumber = current active replicas - // Current active replicas: 1 (MASTER = 1) - Assert.assertEquals(upwardTransitionMsg.getCurrentActiveReplicaNumber(), 1, - "Upward transition should have currentActiveReplicaNumber = current active replica count"); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 1); - // DROPPED transition should not get currentActiveReplicaNumber (downward transition) - Assert.assertEquals(droppedTransitionMsg.getCurrentActiveReplicaNumber(), -1, - "DROPPED transition should not have currentActiveReplicaNumber assigned"); + // Verify: Downward transition gets default value (-1) + Assert.assertEquals(messages.size(), 1); + Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), -1, + "ERROR→OFFLINE transitions should not have currentActiveReplicaNumber assigned (downward)"); } - private ClusterEvent prepareClusterEventForMultiTop(StateModelDefinition stateModelDefinition, - CurrentStateOutput currentStateOutput) { - ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); + // === Tests for DROPPED state handling === - event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput); - event.addAttribute(AttributeName.CURRENT_STATE_EXCLUDING_UNKNOWN.name(), currentStateOutput); + @Test + public void testDroppedReplicasExcludedFromActiveCount() throws Exception { + // Test: DROPPED replicas are excluded from active count calculations + StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - // Mock HelixManager - HelixManager manager = mock(HelixManager.class); - when(manager.getInstanceName()).thenReturn("Controller"); - when(manager.getSessionId()).thenReturn(SESSION_ID); + // Setup: 1 MASTER, 1 OFFLINE, 1 OFFLINE (current active = 1) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "MASTER", INSTANCE_1, "OFFLINE", INSTANCE_2, "OFFLINE"), "MasterSlave"); - // Mock HelixDataAccessor - HelixDataAccessor dataAccessor = mock(HelixDataAccessor.class); - when(manager.getHelixDataAccessor()).thenReturn(dataAccessor); + // Action: One OFFLINE drops, other becomes SLAVE + Map bestPossible = Map.of(INSTANCE_0, "MASTER", // No change + INSTANCE_1, "DROPPED", // OFFLINE -> DROPPED (downward) + INSTANCE_2, "SLAVE" // OFFLINE -> SLAVE (upward) + ); - event.addAttribute(AttributeName.helixmanager.name(), manager); + List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 3); - // Setup ResourceControllerDataProvider - ResourceControllerDataProvider cache = mock(ResourceControllerDataProvider.class); - when(cache.getClusterConfig()).thenReturn(new ClusterConfig("TestCluster")); - when(cache.getStateModelDef("OfflineOnline")).thenReturn(stateModelDefinition); + // Verify: 2 messages, upward gets active count, DROPPED doesn't + Assert.assertEquals(messages.size(), 2); - // Mock live instances - Map liveInstances = new HashMap<>(); - liveInstances.put(INSTANCE_0, createLiveInstance(INSTANCE_0)); - liveInstances.put(INSTANCE_1, createLiveInstance(INSTANCE_1)); - liveInstances.put(INSTANCE_2, createLiveInstance(INSTANCE_2)); - when(cache.getLiveInstances()).thenReturn(liveInstances); + Message upwardMsg = messages.stream() + .filter(m -> m.getTgtName().equals(INSTANCE_2) && m.getToState().equals("SLAVE")) + .findFirst().orElse(null); + Message droppedMsg = messages.stream() + .filter(m -> m.getTgtName().equals(INSTANCE_1) && m.getToState().equals("DROPPED")) + .findFirst().orElse(null); - event.addAttribute(AttributeName.ControllerDataProvider.name(), cache); + Assert.assertNotNull(upwardMsg, "Should have upward transition message"); + Assert.assertNotNull(droppedMsg, "Should have dropped transition message"); - // Setup resources - Map resourceMap = new HashMap<>(); - Resource resource = new Resource(TEST_RESOURCE); - resource.setStateModelDefRef("OfflineOnline"); - resource.addPartition(PARTITION_0); - resourceMap.put(TEST_RESOURCE, resource); - event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), resourceMap); - return event; + Assert.assertEquals(upwardMsg.getCurrentActiveReplicaNumber(), 1, + "Upward transition should have current active replica count"); + Assert.assertEquals(droppedMsg.getCurrentActiveReplicaNumber(), -1, + "DROPPED transition should not have currentActiveReplicaNumber"); } - @Test - public void testReplicaCountCalculationWithDroppedReplicas() throws Exception { - StateModelDefinition stateModelDef = - new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave()); + // === Helper methods === - // Set up current states + /** + * Creates current state map for multiple instances with specified states. + */ + private Map createCurrentStates(Map instanceStates, + String stateModelRef) { Map currentStateMap = new HashMap<>(); + for (Map.Entry entry : instanceStates.entrySet()) { + CurrentState currentState = new CurrentState(TEST_RESOURCE); + currentState.setState(PARTITION_0, entry.getValue()); + currentState.setSessionId(SESSION_ID); + currentState.setStateModelDefRef(stateModelRef); + currentStateMap.put(entry.getKey(), currentState); + } + return currentStateMap; + } - // Instance 0: MASTER -> will stay MASTER - CurrentState currentState0 = new CurrentState(TEST_RESOURCE); - currentState0.setState(PARTITION_0, "MASTER"); - currentState0.setSessionId(SESSION_ID); - currentState0.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_0, currentState0); - - // Instance 1: SLAVE -> will be DROPPED - CurrentState currentState1 = new CurrentState(TEST_RESOURCE); - currentState1.setState(PARTITION_0, "SLAVE"); - currentState1.setSessionId(SESSION_ID); - currentState1.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_1, currentState1); - - // Instance 2: OFFLINE -> will become SLAVE - CurrentState currentState2 = new CurrentState(TEST_RESOURCE); - currentState2.setState(PARTITION_0, "OFFLINE"); - currentState2.setSessionId(SESSION_ID); - currentState2.setStateModelDefRef("MasterSlave"); - currentStateMap.put(INSTANCE_2, currentState2); - - CurrentStateOutput currentStateOutput = setupCurrentStateOutput(currentStateMap); - ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput); - - // Best possible state: MASTER stays, one SLAVE gets DROPPED, one OFFLINE becomes SLAVE - Map partitionMap = new HashMap<>(); - partitionMap.put(INSTANCE_0, "MASTER"); // No change - partitionMap.put(INSTANCE_1, "DROPPED"); // SLAVE -> DROPPED (excluded from active count) - partitionMap.put(INSTANCE_2, "SLAVE"); // OFFLINE -> SLAVE (upward transition) + /** + * Creates CurrentStateOutput from current state map. + */ + private CurrentStateOutput createCurrentStateOutput(Map currentStateMap) { + CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + for (Map.Entry entry : currentStateMap.entrySet()) { + String instance = entry.getKey(); + CurrentState currentState = entry.getValue(); + currentStateOutput.setCurrentState(TEST_RESOURCE, new Partition(PARTITION_0), instance, + currentState.getState(PARTITION_0)); + } + return currentStateOutput; + } - BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); - bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); - event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); + /** + * Processes state transitions and returns generated messages. + */ + private List processAndGetMessages(StateModelDefinition stateModelDef, + Map currentStates, Map bestPossible, int instanceCount) + throws Exception { + CurrentStateOutput currentStateOutput = createCurrentStateOutput(currentStates); + ClusterEvent event = prepareClusterEvent(stateModelDef, currentStateOutput, instanceCount); + setBestPossibleState(event, bestPossible); process(event); MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); - List messages = output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); - - // Should generate 2 messages: SLAVE->DROPPED and OFFLINE->SLAVE - Assert.assertEquals(messages.size(), 2); + return output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + } - Message upwardTransitionMsg = null; - Message droppedTransitionMsg = null; - for (Message msg : messages) { - if (msg.getTgtName().equals(INSTANCE_2)) { - upwardTransitionMsg = msg; - } else if (msg.getTgtName().equals(INSTANCE_1)) { - droppedTransitionMsg = msg; - } - } + /** + * Processes state transitions for OnlineOffline state model. + */ + private List processAndGetMessagesForOnlineOffline(StateModelDefinition stateModelDef, + Map currentStates, Map bestPossible, int instanceCount) + throws Exception { + CurrentStateOutput currentStateOutput = createCurrentStateOutput(currentStates); + ClusterEvent event = + prepareClusterEventForOnlineOffline(stateModelDef, currentStateOutput, instanceCount); + setBestPossibleState(event, bestPossible); - Assert.assertNotNull(upwardTransitionMsg, "Should have upward state transition message"); - Assert.assertNotNull(droppedTransitionMsg, "Should have dropped state transition message"); + process(event); - // Upward transition should get currentActiveReplicaNumber - Assert.assertEquals(upwardTransitionMsg.getCurrentActiveReplicaNumber(), 2, - "Upward state transition message should have current active replica number assigned based on active replica count excluding DROPPED"); + MessageOutput output = event.getAttribute(AttributeName.MESSAGES_ALL.name()); + return output.getMessages(TEST_RESOURCE, new Partition(PARTITION_0)); + } - // DROPPED transition should not get current active replica number (it's a downward transition anyway) - Assert.assertEquals(droppedTransitionMsg.getCurrentActiveReplicaNumber(), -1, - "DROPPED state transition message should not have current active replica number assigned."); + /** + * Sets best possible state in cluster event. + */ + private void setBestPossibleState(ClusterEvent event, Map partitionMap) { + BestPossibleStateOutput bestPossibleOutput = new BestPossibleStateOutput(); + bestPossibleOutput.setState(TEST_RESOURCE, new Partition(PARTITION_0), partitionMap); + event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.name(), bestPossibleOutput); } + /** + * Prepares cluster event with necessary mock objects and configurations. + */ private ClusterEvent prepareClusterEvent(StateModelDefinition stateModelDef, - CurrentStateOutput currentStateOutput) { - ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); - - event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput); - event.addAttribute(AttributeName.CURRENT_STATE_EXCLUDING_UNKNOWN.name(), currentStateOutput); - - // Mock HelixManager - HelixManager manager = mock(HelixManager.class); - when(manager.getInstanceName()).thenReturn("Controller"); - when(manager.getSessionId()).thenReturn(SESSION_ID); - - // Mock HelixDataAccessor - HelixDataAccessor dataAccessor = mock(HelixDataAccessor.class); - when(manager.getHelixDataAccessor()).thenReturn(dataAccessor); - - event.addAttribute(AttributeName.helixmanager.name(), manager); + CurrentStateOutput currentStateOutput, int instanceCount) { + ClusterEvent event = createBaseClusterEvent(currentStateOutput); // Setup ResourceControllerDataProvider ResourceControllerDataProvider cache = mock(ResourceControllerDataProvider.class); - when(cache.getClusterConfig()).thenReturn(new ClusterConfig("TestCluster")); + when(cache.getClusterConfig()).thenReturn(new ClusterConfig(TEST_CLUSTER)); when(cache.getStateModelDef("MasterSlave")).thenReturn(stateModelDef); - - // Mock live instances - Map liveInstances = new HashMap<>(); - liveInstances.put(INSTANCE_0, createLiveInstance(INSTANCE_0)); - liveInstances.put(INSTANCE_1, createLiveInstance(INSTANCE_1)); - liveInstances.put(INSTANCE_2, createLiveInstance(INSTANCE_2)); - liveInstances.put(INSTANCE_3, createLiveInstance(INSTANCE_3)); - when(cache.getLiveInstances()).thenReturn(liveInstances); - + when(cache.getLiveInstances()).thenReturn(createLiveInstances(instanceCount)); event.addAttribute(AttributeName.ControllerDataProvider.name(), cache); // Setup resources - Map resourceMap = new HashMap<>(); - Resource resource = new Resource(TEST_RESOURCE); - resource.setStateModelDefRef("MasterSlave"); - resource.addPartition(PARTITION_0); - resourceMap.put(TEST_RESOURCE, resource); - event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), resourceMap); + event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), + createResourceMap("MasterSlave")); + return event; } - private Map setupCurrentStatesForPrioritizationCase1() { - Map currentStateMap = new HashMap<>(); + /** + * Prepares cluster event for OnlineOffline state model. + */ + private ClusterEvent prepareClusterEventForOnlineOffline(StateModelDefinition stateModelDef, + CurrentStateOutput currentStateOutput, int instanceCount) { + ClusterEvent event = createBaseClusterEvent(currentStateOutput); - // All instances in OFFLINE state - for (String instance : new String[] { - INSTANCE_0, INSTANCE_1, INSTANCE_2, INSTANCE_3 - }) { - CurrentState currentState = new CurrentState(TEST_RESOURCE); - if (instance.equals(INSTANCE_0)) { - currentState.setState(PARTITION_0, "MASTER"); - } else if (instance.equals(INSTANCE_1)) { - currentState.setState(PARTITION_0, "SLAVE"); - } else { - currentState.setState(PARTITION_0, "OFFLINE"); - } - currentState.setSessionId(SESSION_ID); - currentState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(instance, currentState); - } + // Setup ResourceControllerDataProvider for OnlineOffline + ResourceControllerDataProvider cache = mock(ResourceControllerDataProvider.class); + when(cache.getClusterConfig()).thenReturn(new ClusterConfig(TEST_CLUSTER)); + when(cache.getStateModelDef("OfflineOnline")).thenReturn(stateModelDef); + when(cache.getLiveInstances()).thenReturn(createLiveInstances(instanceCount)); + event.addAttribute(AttributeName.ControllerDataProvider.name(), cache); - return currentStateMap; + // Setup resources for OnlineOffline + event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), + createResourceMap("OfflineOnline")); + + return event; } - private Map setupCurrentStatesForPrioritizationCase2() { - Map currentStateMap = new HashMap<>(); + /** + * Creates base cluster event with common attributes. + */ + private ClusterEvent createBaseClusterEvent(CurrentStateOutput currentStateOutput) { + ClusterEvent event = new ClusterEvent(TEST_CLUSTER, ClusterEventType.Unknown); - // All instances in OFFLINE state - for (String instance : new String[] { - INSTANCE_0, INSTANCE_1, INSTANCE_2 - }) { - CurrentState currentState = new CurrentState(TEST_RESOURCE); - currentState.setState(PARTITION_0, "OFFLINE"); - currentState.setSessionId(SESSION_ID); - currentState.setStateModelDefRef("MasterSlave"); - currentStateMap.put(instance, currentState); - } + event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput); + event.addAttribute(AttributeName.CURRENT_STATE_EXCLUDING_UNKNOWN.name(), currentStateOutput); - return currentStateMap; + // Mock HelixManager and HelixDataAccessor + HelixManager manager = mock(HelixManager.class); + when(manager.getInstanceName()).thenReturn("Controller"); + when(manager.getSessionId()).thenReturn(SESSION_ID); + when(manager.getHelixDataAccessor()).thenReturn(mock(HelixDataAccessor.class)); + event.addAttribute(AttributeName.helixmanager.name(), manager); + + return event; } - private CurrentStateOutput setupCurrentStateOutput(Map currentStateMap) { - CurrentStateOutput currentStateOutput = new CurrentStateOutput(); + /** + * Creates mock live instances for the specified count. + */ + private Map createLiveInstances(int instanceCount) { + Map liveInstances = new HashMap<>(); + for (int i = 0; i < instanceCount; i++) { + String instanceName = "localhost_" + i; + ZNRecord znRecord = new ZNRecord(instanceName); + znRecord.setEphemeralOwner(Long.parseLong(SESSION_ID)); - for (Map.Entry entry : currentStateMap.entrySet()) { - String instance = entry.getKey(); - CurrentState currentState = entry.getValue(); + LiveInstance liveInstance = new LiveInstance(znRecord); + liveInstance.setSessionId(SESSION_ID); + liveInstance.setHelixVersion("1.0.0"); + liveInstance.setLiveInstance(instanceName); - currentStateOutput.setCurrentState(TEST_RESOURCE, new Partition(PARTITION_0), instance, - currentState.getState(PARTITION_0)); + liveInstances.put(instanceName, liveInstance); } - - return currentStateOutput; + return liveInstances; } - private LiveInstance createLiveInstance(String instanceName) { - // Create LiveInstance with proper ZNRecord initialization - ZNRecord znRecord = new ZNRecord(instanceName); - znRecord.setEphemeralOwner(Long.parseLong(SESSION_ID)); - - LiveInstance liveInstance = new LiveInstance(znRecord); - liveInstance.setSessionId(SESSION_ID); - liveInstance.setHelixVersion("1.0.0"); - liveInstance.setLiveInstance(instanceName); - - return liveInstance; + /** + * Creates resource map with specified state model reference. + */ + private Map createResourceMap(String stateModelRef) { + Map resourceMap = new HashMap<>(); + Resource resource = new Resource(TEST_RESOURCE); + resource.setStateModelDefRef(stateModelRef); + resource.addPartition(PARTITION_0); + resourceMap.put(TEST_RESOURCE, resource); + return resourceMap; } + /** + * Creates a state transition message. + */ private Message createMessage(String fromState, String toState, String tgtName) { Message message = new Message(Message.MessageType.STATE_TRANSITION, UUID.randomUUID().toString()); From 9044df3431b22fa6bb669ce732313417e84dbe32 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Wed, 18 Jun 2025 10:02:07 -0700 Subject: [PATCH 06/11] fix feedback reviews --- .../stages/MessageGenerationPhase.java | 73 ++++--------------- 1 file changed, 16 insertions(+), 57 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 0fe8233833..e0a7f57317 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -45,7 +45,6 @@ import org.apache.helix.model.IdealState; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; -import org.apache.helix.model.OnlineOfflineSMD; import org.apache.helix.model.Partition; import org.apache.helix.model.Resource; import org.apache.helix.model.ResourceConfig; @@ -70,6 +69,7 @@ public class MessageGenerationPhase extends AbstractBaseStage { .getSystemPropertyAsLong(SystemPropertyKeys.CONTROLLER_MESSAGE_PURGE_DELAY, 60 * 1000); private static final String PENDING_MESSAGE = "pending message"; private static final String STALE_MESSAGE = "stale message"; + private static final String OFFLINE = "OFFLINE"; private static Logger logger = LoggerFactory.getLogger(MessageGenerationPhase.class); @@ -329,10 +329,8 @@ && isTargetActive(nextState, stateModelDef, /** * Calculate the current active replica count based on state model type. - * This method determines how many replicas are currently serving traffic for a partition by - * analyzing the current state distribution and applying state model-specific rules. The count includes - * replicas in top states, secondary top states (where applicable), and ERROR states since helix considers - * them active. + * The count includes replicas in top states, secondary top states (where applicable), + * and ERROR states since helix considers them active. * State model handling: * - Single-top state models: Differentiates between patterns with and without secondary top * states @@ -349,44 +347,16 @@ && isTargetActive(nextState, stateModelDef, */ private int calculateCurrentActiveReplicaCount(Map currentStateMap, StateModelDefinition stateModelDef) { - if (stateModelDef.isSingleTopStateModel()) { - return calculateSingleTopStateActiveCount(currentStateMap, stateModelDef); - } else { - return calculateMultiTopStateActiveCount(currentStateMap, stateModelDef); - } - } - - /** - * Calculate active replica count for single-top state models. - * Single-top state models have different active state definitions: - * - ONLINE-OFFLINE: Only ONLINE (top state) + ERROR are active - * - ONLINE-STANDBY-OFFLINE: ONLINE (top state) + STANDBY (secondary top) + ERROR are active - * Note: We need to identify true secondary states (like STANDBY) vs transition-only - * states (like OFFLINE). - */ - private int calculateSingleTopStateActiveCount(Map currentStateMap, - StateModelDefinition stateModelDef) { List trueSecondaryTopStates = getTrueSecondaryTopStates(stateModelDef); - if (trueSecondaryTopStates.isEmpty()) { - // No true secondary states exist (e.g., ONLINE-OFFLINE pattern) - // Count: top + ERROR states only - // Example: OnlineOffline has getSecondTopStates()=["OFFLINE"] but OFFLINE is non-serving state - // so trueSecondaryTopStates=[] and we only count ONLINE + ERROR - return (int) currentStateMap.values().stream() - .filter(state -> stateModelDef.getTopState().contains(state) - || HelixDefinedState.ERROR.name().equals(state)) - .count(); - } else { - // True secondary states exist (e.g., MASTER-SLAVE, ONLINE-STANDBY, LEADER-STANDBY, OnlineOfflineWithBootstrap) - // Count: top + true secondary top + ERROR states - // Example for MasterSlave: trueSecondaryTopStates=["SLAVE"] - // Example for OnlineOfflineWithBootstrap: trueSecondaryTopStates=["BOOTSTRAP"] - return (int) currentStateMap.values().stream() - .filter(state -> stateModelDef.getTopState().contains(state) - || trueSecondaryTopStates.contains(state) - || HelixDefinedState.ERROR.name().equals(state)) - .count(); - } + return (int) currentStateMap.values().stream() + .filter(state -> stateModelDef.getTopState().contains(state) // Top states (MASTER, ONLINE, + // LEADER) + || trueSecondaryTopStates.contains(state) // True secondary states (SLAVE, STANDBY, + // BOOTSTRAP) + || HelixDefinedState.ERROR.name().equals(state) // ERROR states (still considered + // active) + // DROPPED and OFFLINE are automatically excluded by getTrueSecondaryTopStates() + ).count(); } /** @@ -406,24 +376,13 @@ private int calculateSingleTopStateActiveCount(Map currentStateM */ private List getTrueSecondaryTopStates(StateModelDefinition stateModelDef) { return stateModelDef.getSecondTopStates().stream() - .filter(state -> !stateModelDef.getTopState().equals(state)) // Remove top state duplicates - .filter(state -> !OnlineOfflineSMD.States.OFFLINE.name().equals(state) - && !HelixDefinedState.DROPPED.name().equals(state)) // Remove non-serving states + // Remove top-state duplicates + .filter(state -> !stateModelDef.getTopState().equals(state)) + // Remove non-serving states + .filter(state -> !OFFLINE.equals(state) && !HelixDefinedState.DROPPED.name().equals(state)) .collect(Collectors.toList()); } - /** - * Calculate active replica count for multi-top state models. - * For multi-top state models (e.g., OFFLINE→ONLINE), count only top states + ERROR. - */ - private int calculateMultiTopStateActiveCount(Map currentStateMap, - StateModelDefinition stateModelDef) { - return (int) currentStateMap.values().stream() - .filter(state -> stateModelDef.getTopState().contains(state) - || HelixDefinedState.ERROR.name().equals(state)) - .count(); - } - /** * Determines if the given current state is considered active based on the state model type. * For single-top state models, top, true secondary top, and ERROR states are active. From dedb2e1bf58954076841f4073a31ef91dd0632be Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Wed, 18 Jun 2025 18:26:16 -0700 Subject: [PATCH 07/11] fix feedback reviews --- .../stages/MessageGenerationPhase.java | 92 +++++++------------ 1 file changed, 31 insertions(+), 61 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index e0a7f57317..1d5a17daba 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -166,12 +166,12 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr Map> messageMap = new HashMap<>(); /* - Calculate the current active replica count based on state model type. - This represents the number of replicas currently serving traffic for this partition - Active replicas include: top states, secondary top states (for single-top state models if - they exist)and ERROR states. - All qualifying state transitions for this partition will receive this same value, - allowing clients to understand the current availability level and prioritize accordingly. + * Calculate the current active replica count based on state model type. + * This represents the number of replicas currently serving traffic for this partition + * Active replicas include: top states, secondary top states excluding OFFLINE (if + * they exist) and ERROR states. Active replica count also excluded DROPPED states. + * All qualifying state transitions for this partition will receive this same value, + * allowing clients to understand the current availability level and prioritize accordingly. */ int currentActiveReplicaCount = calculateCurrentActiveReplicaCount(currentStateMap, stateModelDef); @@ -278,10 +278,8 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr - Target state must be considered active (according to state model type) */ if (stateModelDef.isUpwardStateTransition(currentState, nextState) - && !isCurrentlyActive(currentState, stateModelDef, - stateModelDef.isSingleTopStateModel()) - && isTargetActive(nextState, stateModelDef, - stateModelDef.isSingleTopStateModel())) { + && !isStateActive(currentState, stateModelDef) + && isStateActive(nextState, stateModelDef)) { // All qualifying transitions for this partition get the same currentActiveReplicaNumber currentActiveReplicaNumber = currentActiveReplicaCount; @@ -347,86 +345,58 @@ && isTargetActive(nextState, stateModelDef, */ private int calculateCurrentActiveReplicaCount(Map currentStateMap, StateModelDefinition stateModelDef) { - List trueSecondaryTopStates = getTrueSecondaryTopStates(stateModelDef); + List activeSecondaryTopStates = getActiveSecondaryTopStates(stateModelDef); return (int) currentStateMap.values().stream() .filter(state -> stateModelDef.getTopState().contains(state) // Top states (MASTER, ONLINE, // LEADER) - || trueSecondaryTopStates.contains(state) // True secondary states (SLAVE, STANDBY, + || activeSecondaryTopStates.contains(state) // Active secondary states (SLAVE, STANDBY, // BOOTSTRAP) || HelixDefinedState.ERROR.name().equals(state) // ERROR states (still considered // active) - // DROPPED and OFFLINE are automatically excluded by getTrueSecondaryTopStates() + // DROPPED and OFFLINE are automatically excluded by getActiveSecondaryTopStates() ).count(); } /** - * Get true secondary top states - states that: - * 1. Are not the top state itself (avoid double-counting) - * 2. Are not non-serving states like OFFLINE and DROPPED. + * Get active secondary top states - states that are not non-serving states like OFFLINE and DROPPED. * Reasons for elimination: - * - getSecondTopStates() can include the top state itself in some state models. - * Example - OnlineOfflineWithBootstrap: - * topState="ONLINE", getSecondTopStates()=["ONLINE", "BOOTSTRAP"] - * After filtering: trueSecondaryTopStates=["BOOTSTRAP"] (removes "ONLINE" as it is top state.) - * - getSecondTopStates() can also include OFFLINE as a secondary top state in some state models. + * - getSecondTopStates() can include OFFLINE as a secondary top state in some state models. * Example - OnlineOffline: * getSecondTopStates() = ["OFFLINE"] as it transitions to ONLINE. - * After filtering: trueSecondaryTopStates=[] (removes "OFFLINE" as it's not a serving state). + * After filtering: activeSecondaryTopStates=[] (removes "OFFLINE" as it's not a serving state). * @param stateModelDef State model definition containing state hierarchy information */ - private List getTrueSecondaryTopStates(StateModelDefinition stateModelDef) { + private List getActiveSecondaryTopStates(StateModelDefinition stateModelDef) { return stateModelDef.getSecondTopStates().stream() - // Remove top-state duplicates - .filter(state -> !stateModelDef.getTopState().equals(state)) // Remove non-serving states .filter(state -> !OFFLINE.equals(state) && !HelixDefinedState.DROPPED.name().equals(state)) .collect(Collectors.toList()); } /** - * Determines if the given current state is considered active based on the state model type. - * For single-top state models, top, true secondary top, and ERROR states are active. - * For multi-top state models, top and ERROR states are active. - * ERROR state replicas are considered active in HELIX as they do not affect availability. - * @param currentState The current state to check + * Determines if the given state is considered active based on the state model type. + * Active states are defined as: + * - For single-top state models: top states, active secondary top states, and ERROR states + * - For multi-top state models: top states and ERROR states + * ERROR state replicas are always considered active in HELIX as they do not affect + * availability. + * @param state The state to check (can be current state or target state) * @param stateModelDef State model definition containing state hierarchy information - * @param isSingleTopState Whether this is a single-top state model - * @return true if the current state is considered active, false otherwise + * @return true if the state is considered active, false otherwise */ - private boolean isCurrentlyActive(String currentState, StateModelDefinition stateModelDef, - boolean isSingleTopState) { + private boolean isStateActive(String state, StateModelDefinition stateModelDef) { // ERROR state is always considered active regardless of state model type - if (HelixDefinedState.ERROR.name().equals(currentState)) { + if (HelixDefinedState.ERROR.name().equals(state)) { return true; } - if (isSingleTopState) { - return stateModelDef.getTopState().contains(currentState) - || getTrueSecondaryTopStates(stateModelDef).contains(currentState); - } else { - return stateModelDef.getTopState().contains(currentState); - } - } - /** - * Determines if the given target state is considered active based on the state model type. - * For single-top state models, both top,true secondary top and ERROR states are active. - * For multi-top state models, top and ERROR states are active. - * @param targetState The target state to check - * @param stateModelDef State model definition containing state hierarchy information - * @param isSingleTopState Whether this is a single-top state model - * @return true if the target state is considered active, false otherwise - */ - private boolean isTargetActive(String targetState, StateModelDefinition stateModelDef, - boolean isSingleTopState) { - // ERROR state is always considered active regardless of state model type - if (HelixDefinedState.ERROR.name().equals(targetState)) { - return true; - } - if (isSingleTopState) { - return stateModelDef.getTopState().contains(targetState) - || getTrueSecondaryTopStates(stateModelDef).contains(targetState); + if (stateModelDef.isSingleTopStateModel()) { + // For single-top models, both primary top states and active secondary states are considered active + return stateModelDef.getTopState().contains(state) + || getActiveSecondaryTopStates(stateModelDef).contains(state); } else { - return stateModelDef.getTopState().contains(targetState); + // For multi-top models, only top states are considered active + return stateModelDef.getTopState().contains(state); } } From 694f0766515ae0a815b0dc30dde7c207380f3387 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Mon, 23 Jun 2025 13:46:53 -0700 Subject: [PATCH 08/11] feedback fixes - improve comment, consolidate logic for single top and multi top --- .../stages/MessageGenerationPhase.java | 33 +++++-------------- .../TestPrioritizationMessageGeneration.java | 33 +++++++++++++++++-- 2 files changed, 39 insertions(+), 27 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 1d5a17daba..f6a5456828 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -168,8 +168,8 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr /* * Calculate the current active replica count based on state model type. * This represents the number of replicas currently serving traffic for this partition - * Active replicas include: top states, secondary top states excluding OFFLINE (if - * they exist) and ERROR states. Active replica count also excluded DROPPED states. + * Active replicas include: top states, secondary top states(excluding OFFLINE) and ERROR states. + * Active replicas exclude: OFFLINE and DROPPED states. * All qualifying state transitions for this partition will receive this same value, * allowing clients to understand the current availability level and prioritize accordingly. */ @@ -327,15 +327,8 @@ && isStateActive(nextState, stateModelDef)) { /** * Calculate the current active replica count based on state model type. - * The count includes replicas in top states, secondary top states (where applicable), - * and ERROR states since helix considers them active. - * State model handling: - * - Single-top state models: Differentiates between patterns with and without secondary top - * states - * - ONLINE-OFFLINE: Counts ONLINE + ERROR states only - * - MASTER-SLAVE-OFFLINE: Counts MASTER + SLAVE + ERROR states - * - ONLINE-STANDBY-OFFLINE: Counts ONLINE + STANDBY + ERROR states - * - Multi-top state models: Counts only top states + ERROR states + * The count includes replicas in top states, secondary top states (excluding OFFLINE), + * and ERROR states since helix considers them active.Count excludes OFFLINE and DROPPED states. * @param currentStateMap Map of instance name to current state for this partition, representing * the actual state of each replica before any pending transitions * @param stateModelDef State model definition containing the state hierarchy and transition rules @@ -375,11 +368,10 @@ private List getActiveSecondaryTopStates(StateModelDefinition stateModel /** * Determines if the given state is considered active based on the state model type. - * Active states are defined as: - * - For single-top state models: top states, active secondary top states, and ERROR states - * - For multi-top state models: top states and ERROR states - * ERROR state replicas are always considered active in HELIX as they do not affect - * availability. + * Active states include: top states, active secondary top states (excluding OFFLINE), + * and ERROR states. Active states exclude OFFLINE and DROPPED states. + * ERROR state replicas are always considered active in HELIX as they do not + * affect availability. * @param state The state to check (can be current state or target state) * @param stateModelDef State model definition containing state hierarchy information * @return true if the state is considered active, false otherwise @@ -389,15 +381,8 @@ private boolean isStateActive(String state, StateModelDefinition stateModelDef) if (HelixDefinedState.ERROR.name().equals(state)) { return true; } - - if (stateModelDef.isSingleTopStateModel()) { - // For single-top models, both primary top states and active secondary states are considered active - return stateModelDef.getTopState().contains(state) + return stateModelDef.getTopState().contains(state) || getActiveSecondaryTopStates(stateModelDef).contains(state); - } else { - // For multi-top models, only top states are considered active - return stateModelDef.getTopState().contains(state); - } } private boolean shouldCreateSTCancellation(Message pendingMessage, String desiredState, diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index 1e6369e15e..f809a65a96 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -32,6 +32,7 @@ import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; import org.apache.helix.model.OnlineOfflineSMD; +import org.apache.helix.model.OnlineOfflineWithBootstrapSMD; import org.apache.helix.model.Partition; import org.apache.helix.model.Resource; import org.apache.helix.model.StateModelDefinition; @@ -243,8 +244,7 @@ public void testSingleTopStateModelWithSecondaryTop() throws Exception { } @Test - public void testMultiTopStateModel() throws Exception { - // Test: Multi-top state model - only top states + ERROR count as active + public void testMultiTopStateModelWithoutSecondaryTop() throws Exception { StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForOnlineOffline()); // Setup: 1 ONLINE, 2 OFFLINE (current active = 1) @@ -265,7 +265,34 @@ public void testMultiTopStateModel() throws Exception { Assert.assertEquals(messages.size(), 2); for (Message msg : messages) { Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 1, - "Multi-top state model: only top states count as active"); + "Multi-top state model without secondary top: only top states count as active"); + } + } + + @Test + public void testMultiTopStateModelWithSecondaryTop() throws Exception { + StateModelDefinition stateModelDef = OnlineOfflineWithBootstrapSMD.build(); + + // Setup: 2 ONLINE, 1 BOOTSTRAP, 1 OFFLINE (current active = 3) + Map currentStates = createCurrentStates( + Map.of(INSTANCE_0, "ONLINE", INSTANCE_1, "ONLINE", INSTANCE_2, "BOOTSTRAP", INSTANCE_3, "OFFLINE"), + "OnlineOfflineWithBootstrap"); + + // Action: BOOTSTRAP becomes ONLINE and OFFLINE becomes BOOTSTRAP. + Map bestPossible = Map.of(INSTANCE_0, "ONLINE", // No change + INSTANCE_1, "ONLINE", // No change + INSTANCE_2, "BOOTSTRAP", // No change + INSTANCE_3, "BOOTSTRAP" // OFFLINE -> BOOTSTRAP (upward) + ); + + List messages = + processAndGetMessagesForOnlineOffline(stateModelDef, currentStates, bestPossible, 4); + + // Verify: Current active = 3 (ONLINE + BOOTSTRAP state counts) + Assert.assertEquals(messages.size(), 1); + for (Message msg : messages) { + Assert.assertEquals(msg.getCurrentActiveReplicaNumber(), 3, + "Multi-top state model without secondary top: top states and secondary top states count as active"); } } From bb5423a3a757f0e4cf17df6ff10f1b5f37c3f7f3 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Mon, 23 Jun 2025 13:53:30 -0700 Subject: [PATCH 09/11] update a stale comment --- .../controller/stages/TestPrioritizationMessageGeneration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index f809a65a96..6fe1184cb4 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -278,7 +278,7 @@ public void testMultiTopStateModelWithSecondaryTop() throws Exception { Map.of(INSTANCE_0, "ONLINE", INSTANCE_1, "ONLINE", INSTANCE_2, "BOOTSTRAP", INSTANCE_3, "OFFLINE"), "OnlineOfflineWithBootstrap"); - // Action: BOOTSTRAP becomes ONLINE and OFFLINE becomes BOOTSTRAP. + // Action: OFFLINE becomes BOOTSTRAP. Map bestPossible = Map.of(INSTANCE_0, "ONLINE", // No change INSTANCE_1, "ONLINE", // No change INSTANCE_2, "BOOTSTRAP", // No change From 003bfc522393820809c609976dddda2b3bf458f8 Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Wed, 25 Jun 2025 14:28:03 -0700 Subject: [PATCH 10/11] create custom state model class for test, fix alignment issues and remove filtering out second top state to top state --- .../stages/MessageGenerationPhase.java | 42 +++++----- .../apache/helix/model/OnlineOfflineSMD.java | 21 ++--- .../TestPrioritizationMessageGeneration.java | 79 +++++++++++++------ 3 files changed, 82 insertions(+), 60 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index f6a5456828..5c02efca10 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -262,34 +262,33 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr pendingMessage, manager, resource, partition, sessionIdMap, instanceName, stateModelDef, cancellationMessage, isCancellationEnabled); } else { - // Set currentActiveReplicaNumber to provide metadata for potential message prioritization by - // participant. + // Set currentActiveReplicaNumber to provide metadata for potential message + // prioritization by participant. // Assign the current active replica count to all qualifying upward transitions for this // partition. // This ensures consistent prioritization metadata across concurrent state transitions. - int currentActiveReplicaNumber = -1; // -1 indicates no prioritization metadata, for eg: - // Downward ST messages get a -1. + // -1 indicates no prioritization metadata, for eg:Downward ST messages get a -1. + int currentActiveReplicaNumber = -1; /* - Assign currentActiveReplicaNumber for qualifying upward state transitions. - Criteria for assignment: - - Must be an upward state transition according to state model - - Current state must not be considered active (according to state model type) - - Target state must be considered active (according to state model type) + * Assign currentActiveReplicaNumber for qualifying upward state transitions. + * Criteria for assignment: + * - Must be an upward state transition according to state model + * - Target state must be considered active (according to state model type) */ if (stateModelDef.isUpwardStateTransition(currentState, nextState) - && !isStateActive(currentState, stateModelDef) && isStateActive(nextState, stateModelDef)) { - // All qualifying transitions for this partition get the same currentActiveReplicaNumber + // All qualifying transitions for this partition get the same + // currentActiveReplicaNumber currentActiveReplicaNumber = currentActiveReplicaCount; } // Create new state transition message - message = MessageUtil - .createStateTransitionMessage(manager.getInstanceName(), manager.getSessionId(), - resource, partition.getPartitionName(), instanceName, currentState, nextState, - sessionIdMap.get(instanceName), stateModelDef.getId(), currentActiveReplicaNumber); + message = MessageUtil.createStateTransitionMessage(manager.getInstanceName(), + manager.getSessionId(), resource, partition.getPartitionName(), instanceName, + currentState, nextState, sessionIdMap.get(instanceName), stateModelDef.getId(), + currentActiveReplicaNumber); if (logger.isDebugEnabled()) { LogUtil.logDebug(logger, _eventId, String.format( @@ -338,12 +337,12 @@ && isStateActive(nextState, stateModelDef)) { */ private int calculateCurrentActiveReplicaCount(Map currentStateMap, StateModelDefinition stateModelDef) { - List activeSecondaryTopStates = getActiveSecondaryTopStates(stateModelDef); return (int) currentStateMap.values().stream() .filter(state -> stateModelDef.getTopState().contains(state) // Top states (MASTER, ONLINE, // LEADER) - || activeSecondaryTopStates.contains(state) // Active secondary states (SLAVE, STANDBY, - // BOOTSTRAP) + || getActiveSecondaryTopStates(stateModelDef).contains(state) // Active secondary states + // (SLAVE, STANDBY, + // BOOTSTRAP) || HelixDefinedState.ERROR.name().equals(state) // ERROR states (still considered // active) // DROPPED and OFFLINE are automatically excluded by getActiveSecondaryTopStates() @@ -351,7 +350,8 @@ private int calculateCurrentActiveReplicaCount(Map currentStateM } /** - * Get active secondary top states - states that are not non-serving states like OFFLINE and DROPPED. + * Get active secondary top states - states that are not non-serving states like OFFLINE and + * DROPPED. * Reasons for elimination: * - getSecondTopStates() can include OFFLINE as a secondary top state in some state models. * Example - OnlineOffline: @@ -381,8 +381,8 @@ private boolean isStateActive(String state, StateModelDefinition stateModelDef) if (HelixDefinedState.ERROR.name().equals(state)) { return true; } - return stateModelDef.getTopState().contains(state) - || getActiveSecondaryTopStates(stateModelDef).contains(state); + return stateModelDef.getTopState().contains(state) + || getActiveSecondaryTopStates(stateModelDef).contains(state); } private boolean shouldCreateSTCancellation(Message pendingMessage, String desiredState, diff --git a/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java b/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java index ab207218ac..fd97c7ba9f 100644 --- a/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java +++ b/helix-core/src/main/java/org/apache/helix/model/OnlineOfflineSMD.java @@ -42,21 +42,11 @@ public OnlineOfflineSMD() { } /** - * Build OnlineOffline state model definition with default replica count - * @return StateModelDefinition for OnlineOffline model + * Build OnlineOffline state model definition + * @return */ public static StateModelDefinition build() { - return build(StateModelDefinition.STATE_REPLICA_COUNT_ALL_REPLICAS); - } - - /** - * Build OnlineOffline state model definition with custom replica count - * @param instanceCount the maximum number of instances that can be in ONLINE state - * @return StateModelDefinition for OnlineOffline model - */ - public static StateModelDefinition build(String instanceCount) { - StateModelDefinition.Builder builder = new StateModelDefinition.Builder(name); - + StateModelDefinition.Builder builder =new StateModelDefinition.Builder(name); // init state builder.initialState(States.OFFLINE.name()); @@ -72,8 +62,9 @@ public static StateModelDefinition build(String instanceCount) { builder.addTransition(States.OFFLINE.name(), States.ONLINE.name(), 1); builder.addTransition(States.OFFLINE.name(), HelixDefinedState.DROPPED.name()); - // bounds - uses the instanceCount parameter instead of constant - builder.dynamicUpperBound(States.ONLINE.name(), instanceCount); + // bounds + builder.dynamicUpperBound(States.ONLINE.name(), + StateModelDefinition.STATE_REPLICA_COUNT_ALL_REPLICAS); return builder.build(); } diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java index 6fe1184cb4..1a9b50ae3e 100644 --- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java +++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestPrioritizationMessageGeneration.java @@ -25,13 +25,13 @@ import java.util.UUID; import org.apache.helix.HelixDataAccessor; +import org.apache.helix.HelixDefinedState; import org.apache.helix.HelixManager; import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider; import org.apache.helix.model.ClusterConfig; import org.apache.helix.model.CurrentState; import org.apache.helix.model.LiveInstance; import org.apache.helix.model.Message; -import org.apache.helix.model.OnlineOfflineSMD; import org.apache.helix.model.OnlineOfflineWithBootstrapSMD; import org.apache.helix.model.Partition; import org.apache.helix.model.Resource; @@ -131,25 +131,6 @@ public void testNoReplicaNumberForNonUpwardTransitions() throws Exception { "Non-upward state transitions should not have currentActiveReplicaNumber assigned"); } - @Test - public void testNoReplicaNumberForSecondTopToTopTransitions() throws Exception { - // Test: SLAVE -> MASTER (second top to top) should not receive currentActiveReplicaNumber - StateModelDefinition stateModelDef = new StateModelDefinition(generateConfigForMasterSlave()); - - Map currentStates = - createCurrentStates(Map.of(INSTANCE_0, "SLAVE"), "MasterSlave"); - - // Action: SLAVE -> MASTER (second top to top transition) - Map bestPossible = Map.of(INSTANCE_0, "MASTER"); - - List messages = processAndGetMessages(stateModelDef, currentStates, bestPossible, 1); - - // Verify: Second top to top transition gets default value (-1) - Assert.assertEquals(messages.size(), 1); - Assert.assertEquals(messages.get(0).getCurrentActiveReplicaNumber(), -1, - "Second top to top state transitions should not have currentActiveReplicaNumber assigned"); - } - // === Tests for pending messages === @Test @@ -194,7 +175,7 @@ public void testPendingMessagesDoNotAffectCurrentReplicaCount() throws Exception @Test public void testSingleTopStateModelWithoutSecondaryTop() throws Exception { // Test: ONLINE-OFFLINE model (single top without secondary) - only top + ERROR count as active - StateModelDefinition onlineOfflineStateModel = OnlineOfflineSMD.build("1"); + StateModelDefinition onlineOfflineStateModel = CustomOnlineOfflineSMD.build(1); // Verify this is a single-top state model Assert.assertTrue(onlineOfflineStateModel.isSingleTopStateModel(), @@ -274,9 +255,9 @@ public void testMultiTopStateModelWithSecondaryTop() throws Exception { StateModelDefinition stateModelDef = OnlineOfflineWithBootstrapSMD.build(); // Setup: 2 ONLINE, 1 BOOTSTRAP, 1 OFFLINE (current active = 3) - Map currentStates = createCurrentStates( - Map.of(INSTANCE_0, "ONLINE", INSTANCE_1, "ONLINE", INSTANCE_2, "BOOTSTRAP", INSTANCE_3, "OFFLINE"), - "OnlineOfflineWithBootstrap"); + Map currentStates = + createCurrentStates(Map.of(INSTANCE_0, "ONLINE", INSTANCE_1, "ONLINE", INSTANCE_2, + "BOOTSTRAP", INSTANCE_3, "OFFLINE"), "OnlineOfflineWithBootstrap"); // Action: OFFLINE becomes BOOTSTRAP. Map bestPossible = Map.of(INSTANCE_0, "ONLINE", // No change @@ -560,4 +541,54 @@ private Message createMessage(String fromState, String toState, String tgtName) message.setSrcSessionId(SESSION_ID); return message; } + + /** + * Custom OnlineOffline state model with configurable upper bounds for ONLINE state. + * Enables testing scenarios with specific replica count constraints. + */ + private static final class CustomOnlineOfflineSMD { + private static final String STATE_MODEL_NAME = "CustomOnlineOffline"; + + /** + * States for the CustomOnlineOffline state model + */ + private enum States { + ONLINE, + OFFLINE + } + + /** + * Build OnlineOffline state model definition with custom instance count + * @param instanceCount the maximum number of instances that can be in ONLINE state + * @return StateModelDefinition for OnlineOffline model with custom bounds + */ + public static StateModelDefinition build(int instanceCount) { + if (instanceCount <= 0) { + throw new IllegalArgumentException( + "Instance count must be positive, got: " + instanceCount); + } + + StateModelDefinition.Builder builder = new StateModelDefinition.Builder(STATE_MODEL_NAME); + + // init state + builder.initialState(States.OFFLINE.name()); + + // add states + builder.addState(States.ONLINE.name(), 0); + builder.addState(States.OFFLINE.name(), 1); + for (final HelixDefinedState state : HelixDefinedState.values()) { + builder.addState(state.name()); + } + + // add transitions + builder.addTransition(States.ONLINE.name(), States.OFFLINE.name(), 0); + builder.addTransition(States.OFFLINE.name(), States.ONLINE.name(), 1); + builder.addTransition(States.OFFLINE.name(), HelixDefinedState.DROPPED.name()); + + // bounds - uses the instanceCount parameter + builder.dynamicUpperBound(States.ONLINE.name(), String.valueOf(instanceCount)); + + return builder.build(); + } + } } From 40d5ac32eabd4ce651248b4ca7c18e5d1ce791ca Mon Sep 17 00:00:00 2001 From: Charanya Sudharsanan Date: Thu, 26 Jun 2025 12:00:37 -0700 Subject: [PATCH 11/11] format and code clean --- .../stages/MessageGenerationPhase.java | 18 +++++++++--------- .../java/org/apache/helix/model/Message.java | 14 +++++++------- .../helix/model/StateModelDefinition.java | 5 ----- 3 files changed, 16 insertions(+), 21 deletions(-) diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java index 5c02efca10..0868ac6d6a 100644 --- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java +++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java @@ -168,12 +168,14 @@ private void generateMessage(final Resource resource, final BaseControllerDataPr /* * Calculate the current active replica count based on state model type. * This represents the number of replicas currently serving traffic for this partition - * Active replicas include: top states, secondary top states(excluding OFFLINE) and ERROR states. + * Active replicas include: top states, secondary top states(excluding OFFLINE) and ERROR + * states. * Active replicas exclude: OFFLINE and DROPPED states. * All qualifying state transitions for this partition will receive this same value, * allowing clients to understand the current availability level and prioritize accordingly. */ - int currentActiveReplicaCount = calculateCurrentActiveReplicaCount(currentStateMap, stateModelDef); + int currentActiveReplicaCount = + calculateCurrentActiveReplicaCount(currentStateMap, stateModelDef); for (String instanceName : instanceStateMap.keySet()) { @@ -328,10 +330,8 @@ && isStateActive(nextState, stateModelDef)) { * Calculate the current active replica count based on state model type. * The count includes replicas in top states, secondary top states (excluding OFFLINE), * and ERROR states since helix considers them active.Count excludes OFFLINE and DROPPED states. - * @param currentStateMap Map of instance name to current state for this partition, representing - * the actual state of each replica before any pending transitions - * @param stateModelDef State model definition containing the state hierarchy and transition rules - * used to determine which states are considered active + * @param currentStateMap + * @param stateModelDef * @return The number of replicas currently in active states, used to determine the * currentActiveReplicaNumber for the partition. */ @@ -357,7 +357,7 @@ private int calculateCurrentActiveReplicaCount(Map currentStateM * Example - OnlineOffline: * getSecondTopStates() = ["OFFLINE"] as it transitions to ONLINE. * After filtering: activeSecondaryTopStates=[] (removes "OFFLINE" as it's not a serving state). - * @param stateModelDef State model definition containing state hierarchy information + * @param stateModelDef */ private List getActiveSecondaryTopStates(StateModelDefinition stateModelDef) { return stateModelDef.getSecondTopStates().stream() @@ -372,8 +372,8 @@ private List getActiveSecondaryTopStates(StateModelDefinition stateModel * and ERROR states. Active states exclude OFFLINE and DROPPED states. * ERROR state replicas are always considered active in HELIX as they do not * affect availability. - * @param state The state to check (can be current state or target state) - * @param stateModelDef State model definition containing state hierarchy information + * @param state + * @param stateModelDef * @return true if the state is considered active, false otherwise */ private boolean isStateActive(String state, StateModelDefinition stateModelDef) { diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java index a2a0c7be99..68751de40d 100644 --- a/helix-core/src/main/java/org/apache/helix/model/Message.java +++ b/helix-core/src/main/java/org/apache/helix/model/Message.java @@ -938,18 +938,18 @@ public void setSrcClusterName(String clusterName) { * for this partition at the time the state transition message is generated. * Active states include top states, secondary top states (for single-top state models), * and ERROR states. - * * This metadata enables participants to prioritize recovery scenarios (low active counts) * over load balancing scenarios (high active counts) in custom thread pools or message handlers. * For example, 2→3 transitions get higher priority than 3→4 transitions. - * - * Default value is -1 for transitions that don't require prioritization metadata.(for eg : downward transitions). - * - * @param currentActiveReplicaNumber the number of currently active replicas (-1 when there is no prioritization metadata, - * >=0 for transitions containing current availability level) + * Default value is -1 for transitions that don't require prioritization metadata.(for eg : + * downward transitions). + * @param currentActiveReplicaNumber the number of currently active replicas (-1 when there is no + * prioritization metadata, + * >=0 for transitions containing current availability level) */ public void setCurrentActiveReplicaNumber(int currentActiveReplicaNumber) { - _record.setIntField(Attributes.CURRENT_ACTIVE_REPLICA_NUMBER.name(), currentActiveReplicaNumber); + _record.setIntField(Attributes.CURRENT_ACTIVE_REPLICA_NUMBER.name(), + currentActiveReplicaNumber); } /** diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java index becccfa943..e79d589384 100644 --- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java +++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java @@ -513,11 +513,6 @@ public static Map getStateCounts(Map stateMap) * @return True if it's an upward state transition, false otherwise */ public boolean isUpwardStateTransition(String fromState, String toState) { - - if (fromState == null || toState == null) { - return false; - } - Map statePriorityMap = getStatePriorityMap(); Integer fromStateWeight = statePriorityMap.get(fromState);