You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2018/03/27 23:22:02 UTC

helix git commit: [HELIX-686] Add support for WorkflowContext in PropertyKey

Repository: helix
Updated Branches:
  refs/heads/master 7c8a271c1 -> ccdc0dd7f


[HELIX-686] Add support for WorkflowContext in PropertyKey


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/ccdc0dd7
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/ccdc0dd7
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/ccdc0dd7

Branch: refs/heads/master
Commit: ccdc0dd7f49e32a731b6fbb5a088a52ced021df4
Parents: 7c8a271
Author: narendly <na...@gmail.com>
Authored: Mon Mar 26 14:34:01 2018 -0700
Committer: narendly <na...@gmail.com>
Committed: Tue Mar 27 16:21:11 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/helix/PropertyKey.java | 15 ++++-
 .../org/apache/helix/PropertyPathBuilder.java   | 11 +++-
 .../java/org/apache/helix/PropertyType.java     | 10 +++-
 .../helix/util/TestGetWorkflowContext.java      | 58 ++++++++++++++++++++
 4 files changed, 88 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ccdc0dd7/helix-core/src/main/java/org/apache/helix/PropertyKey.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
index 4e58a89..a5302f8 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -39,6 +39,7 @@ import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.StatusUpdate;
+import org.apache.helix.task.WorkflowContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -312,7 +313,7 @@ public class PropertyKey {
     public PropertyKey instances() {
       return new PropertyKey(PropertyType.INSTANCES, null, _clusterName);
     }
-    
+
     /**
      * Get a property key associated with specified instance
      * @return {@link PropertyKey}
@@ -700,6 +701,16 @@ public class PropertyKey {
     public PropertyKey healthReports(String instanceName) {
       return new PropertyKey(PropertyType.HEALTHREPORT, HealthStat.class, _clusterName, instanceName);
     }
+
+    /**
+     * Get a property key associated with {@link WorkflowContext}
+     * TODO: Below must handle the case for future versions of Task Framework with a different path structure
+     * @param workflowName
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey workflowContext(String workflowName) {
+      return new PropertyKey(PropertyType.WORKFLOWCONTEXT, WorkflowContext.class, _clusterName, workflowName);
+    }
   }
 
   /**
@@ -734,4 +745,4 @@ public class PropertyKey {
     return _configScope;
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/helix/blob/ccdc0dd7/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java b/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
index cfff5bb..b0dacdb 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
@@ -36,11 +36,13 @@ import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.StatusUpdate;
+import org.apache.helix.task.WorkflowContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.helix.PropertyType.*;
 
+
 /**
  * Utility mapping properties to their Zookeeper locations
  */
@@ -63,6 +65,8 @@ public class PropertyPathBuilder {
     typeToClassMapping.put(HISTORY, LeaderHistory.class);
     typeToClassMapping.put(PAUSE, PauseSignal.class);
     typeToClassMapping.put(MAINTENANCE, MaintenanceSignal.class);
+    // TODO: Below must handle the case for future versions of Task Framework with a different path structure
+    typeToClassMapping.put(WORKFLOWCONTEXT, WorkflowContext.class);
 
     // @formatter:off
     addEntry(PropertyType.CONFIGS, 1, "/{clusterName}/CONFIGS");
@@ -128,6 +132,11 @@ public class PropertyPathBuilder {
     addEntry(PropertyType.MAINTENANCE, 1, "/{clusterName}/CONTROLLER/MAINTENANCE");
     // @formatter:on
 
+    // RESOURCE
+    // TODO: Below must handle the case for future versions of Task Framework with a different path structure
+    addEntry(PropertyType.WORKFLOWCONTEXT, 2,
+        "/{clusterName}/PROPERTYSTORE/TaskRebalancer/{workflowName}/Context");
+
   }
   static Pattern pattern = Pattern.compile("(\\{.+?\\})");
 
@@ -352,4 +361,4 @@ public class PropertyPathBuilder {
   public static String maintenance(String clusterName) {
     return String.format("/%s/CONTROLLER/MAINTENANCE", clusterName);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/helix/blob/ccdc0dd7/helix-core/src/main/java/org/apache/helix/PropertyType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyType.java b/helix-core/src/main/java/org/apache/helix/PropertyType.java
index 73da39e..261d449 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyType.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyType.java
@@ -26,7 +26,8 @@ enum Type {
   CLUSTER,
   INSTANCE,
   CONTROLLER,
-  RESOURCE
+  RESOURCE,
+  TASK
 }
 
 /**
@@ -61,7 +62,10 @@ public enum PropertyType {
   MAINTENANCE(Type.CONTROLLER, true, false, true),
   MESSAGES_CONTROLLER(Type.CONTROLLER, true, false, true),
   STATUSUPDATES_CONTROLLER(Type.CONTROLLER, true, true, true),
-  ERRORS_CONTROLLER(Type.CONTROLLER, true, true, true);
+  ERRORS_CONTROLLER(Type.CONTROLLER, true, true, true),
+
+  // TASK PROPERTY
+  WORKFLOWCONTEXT(Type.TASK, true, false, false, false, false);
 
   // @formatter:on
 
@@ -197,4 +201,4 @@ public enum PropertyType {
   public boolean isCached() {
     return isCached;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/helix/blob/ccdc0dd7/helix-core/src/test/java/org/apache/helix/util/TestGetWorkflowContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/util/TestGetWorkflowContext.java b/helix-core/src/test/java/org/apache/helix/util/TestGetWorkflowContext.java
new file mode 100644
index 0000000..890e6c0
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/util/TestGetWorkflowContext.java
@@ -0,0 +1,58 @@
+package org.apache.helix.util;
+
+/*
+ * 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 com.google.common.base.Joiner;
+import org.apache.helix.AccessOption;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.integration.task.TaskTestBase;
+import org.apache.helix.task.TaskConstants;
+import org.apache.helix.task.WorkflowContext;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class TestGetWorkflowContext extends TaskTestBase {
+
+  private static final String WORKFLOW_NAME = "testWorkflow_01";
+  private static final String CONTEXT_NODE = "Context";
+
+  /**
+   * This test method tests whether PropertyKey.Builder successfully creates a path for WorkflowContext instances.
+   * TODO: KeyBuilder must handle the case for future versions of Task Framework with a different path structure
+   */
+  @Test
+  public void testGetWorkflowContext() {
+    // Manually create a WorkflowContext instance
+    ZNRecord znRecord = new ZNRecord(WORKFLOW_NAME);
+    WorkflowContext workflowContext = new WorkflowContext(znRecord);
+    _manager.getHelixPropertyStore().set(
+        Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, WORKFLOW_NAME, CONTEXT_NODE),
+        workflowContext.getRecord(), AccessOption.PERSISTENT);
+
+    // Test retrieving this WorkflowContext using PropertyKey.Builder.getPath()
+    PropertyKey.Builder keyBuilder = new PropertyKey.Builder(CLUSTER_NAME);
+    String path = keyBuilder.workflowContext(WORKFLOW_NAME).getPath();
+    WorkflowContext workflowCtx = new WorkflowContext(_baseAccessor.get(path, null, AccessOption.PERSISTENT));
+
+    Assert.assertEquals(workflowContext, workflowCtx);
+  }
+}
\ No newline at end of file