You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2021/06/25 00:50:51 UTC

[GitHub] [helix] jiajunwang commented on a change in pull request #1803: Add message generation logic for management pipeline

jiajunwang commented on a change in pull request #1803:
URL: https://github.com/apache/helix/pull/1803#discussion_r658374766



##########
File path: helix-core/src/main/java/org/apache/helix/api/status/ClusterManagementMode.java
##########
@@ -67,4 +69,26 @@ public Status getStatus() {
     public Type getMode() {
         return mode;
     }
+
+    public boolean isFullyInNormalMode() {

Review comment:
       Question, do we have a term for partially normal status, or what does that stand for?
   
   I assume it should be,
   1. NORMAL, COMPLETE,
   2. CLUSTER_PAUSE, IN_PROGRESS
   3. CLUSTER_PAUSE, COMPLETE
   4. CLUSTER_PAUSE, IN_PROGRESS ---> it is recovering from pause, but since it is still partially paused, I think to mark it CLUSTER_PAUSE is more appropriate.
   5. NORMAL, COMPLETE

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/ManagementMessageDispatchStage.java
##########
@@ -0,0 +1,62 @@
+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.List;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.status.ClusterManagementMode;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.dataproviders.ManagementControllerDataProvider;
+import org.apache.helix.model.Message;
+import org.apache.helix.util.RebalanceUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dispatches participant status change and pending state transition cancellation messages
+ * for the management pipeline.
+ */
+public class ManagementMessageDispatchStage extends MessageDispatchStage {
+  private static final Logger LOG = LoggerFactory.getLogger(ManagementMessageDispatchStage.class);
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    MessageOutput messageOutput =
+        event.getAttribute(AttributeName.MESSAGES_ALL.name());
+    processEvent(event, messageOutput);
+
+    // Send participant status change messages.
+    HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+    List<Message> messagesSent =
+        super.sendMessages(manager.getHelixDataAccessor(), messageOutput.getStatusChangeMessages());
+    ManagementControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+    cache.cacheMessages(messagesSent);
+
+    // Can exit management mode pipeline after fully in normal mode
+    ClusterManagementMode managementMode = event.getAttribute(AttributeName.CLUSTER_STATUS.name());
+    if (managementMode.isFullyInNormalMode()) {

Review comment:
       Why it is determined here instead of ManagementModeStage?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/ManagementMessageGenerationPhase.java
##########
@@ -0,0 +1,118 @@
+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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.status.ClusterManagementMode;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.dataproviders.ManagementControllerDataProvider;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.LiveInstance.LiveInstanceStatus;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.PauseSignal;
+import org.apache.helix.util.MessageUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Generates participant status change (freeze/unfreeze) and pending state transition cancellation
+ * messages for management mode pipeline.
+ */
+public class ManagementMessageGenerationPhase extends MessageGenerationPhase {
+  private static final Logger LOG = LoggerFactory.getLogger(ManagementMessageGenerationPhase.class);
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+    String clusterName = event.getClusterName();
+    HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+    ClusterManagementMode managementMode = event.getAttribute(AttributeName.CLUSTER_STATUS.name());
+    ManagementControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+    if (manager == null || managementMode == null || cache == null) {
+      throw new StageException("Missing attributes in event: " + event
+          + ". Requires HelixManager|ClusterStatus|DataCache");
+    }
+
+    PauseSignal pauseSignal = cache.getPauseSignal();
+    if (cache.getClusterConfig().isStateTransitionCancelEnabled()
+        && pauseSignal != null && pauseSignal.getCancelPendingST()) {
+      // Generate ST cancellation messages.
+      LogUtil.logInfo(LOG, _eventId,
+          "Generating ST cancellation messages for cluster " + clusterName);
+      super.process(event);

Review comment:
       One suggestion, put the current state to best possible state transformation here so it won't be done unless necessary.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/ManagementModeStage.java
##########
@@ -66,6 +68,14 @@ public void process(ClusterEvent event) throws Exception {
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     ManagementControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
+    CurrentStateOutput currentStateOutput =
+        event.getAttribute(AttributeName.CURRENT_STATE.name());
+    final Map<String, Resource> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+
+    final BestPossibleStateOutput bestPossibleStateOutput =

Review comment:
       I assume this is only used to cancel pending ST. Calculate it here might be confusing because it is very far away from where it is used.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org