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/11/14 19:02:20 UTC

[2/2] helix git commit: [HELIX-790] REST2.0: Add support for updating IdealState

[HELIX-790] REST2.0: Add support for updating IdealState

There was a user request for a REST endpoint that allows users to add/delete/modify fields in IdealState ZNodes.
Changelist:
1. Add updateResourceIdealState in ResourceAcessor
2. Add update APIs in HelixAdmin
3. Add an integration test


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

Branch: refs/heads/master
Commit: abc6969d754e01c76278c266d08cc4e9fb80e910
Parents: 22fa03f
Author: narendly <na...@gmail.com>
Authored: Tue Nov 13 18:22:55 2018 -0800
Committer: narendly <na...@gmail.com>
Committed: Tue Nov 13 18:22:55 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/helix/HelixAdmin.java  | 16 ++++
 .../apache/helix/manager/zk/ZKHelixAdmin.java   | 34 ++++++++
 .../org/apache/helix/mock/MockHelixAdmin.java   | 10 +++
 .../rest/server/resources/AbstractResource.java |  7 +-
 .../resources/helix/ResourceAccessor.java       | 48 +++++++++++
 .../helix/rest/server/TestResourceAccessor.java | 88 +++++++++++++++++++-
 6 files changed, 198 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
index 9562a0b..5cb8883 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
@@ -209,6 +209,22 @@ public interface HelixAdmin {
   void setResourceIdealState(String clusterName, String resourceName, IdealState idealState);
 
   /**
+   * Selectively updates fields for an existing resource's IdealState ZNode.
+   * @param clusterName
+   * @param resourceName
+   * @param idealState
+   */
+  void updateIdealState(String clusterName, String resourceName, IdealState idealState);
+
+  /**
+   * Selectively removes fields for an existing resource's IdealState ZNode.
+   * @param clusterName
+   * @param resourceName
+   * @param idealState
+   */
+  void removeFromIdealState(String clusterName, String resourceName, IdealState idealState);
+
+  /**
    * Disable or enable an instance
    * @param clusterName
    * @param instanceName

http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index 0f79175..ad8a9a2 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -819,6 +819,40 @@ public class ZKHelixAdmin implements HelixAdmin {
     accessor.setProperty(keyBuilder.idealStates(resourceName), idealState);
   }
 
+  /**
+   * Partially updates the fields appearing in the given IdealState (input).
+   * @param clusterName
+   * @param resourceName
+   * @param idealState
+   */
+  @Override
+  public void updateIdealState(String clusterName, String resourceName, IdealState idealState) {
+    if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) {
+      throw new HelixException(
+          "updateIdealState failed. Cluster: " + clusterName + " is NOT setup properly.");
+    }
+    String zkPath = PropertyPathBuilder.idealState(clusterName, resourceName);
+    if (!_zkClient.exists(zkPath)) {
+      throw new HelixException(String.format(
+          "updateIdealState failed. The IdealState for the given resource does not already exist. Resource name: %s",
+          resourceName));
+    }
+    // Update by way of merge
+    ZKUtil.createOrUpdate(_zkClient, zkPath, idealState.getRecord(), true, true);
+  }
+
+  /**
+   * Selectively removes fields appearing in the given IdealState (input) from the IdealState in ZK.
+   * @param clusterName
+   * @param resourceName
+   * @param idealState
+   */
+  @Override
+  public void removeFromIdealState(String clusterName, String resourceName, IdealState idealState) {
+    String zkPath = PropertyPathBuilder.idealState(clusterName, resourceName);
+    ZKUtil.subtract(_zkClient, zkPath, idealState.getRecord());
+  }
+
   @Override
   public ExternalView getResourceExternalView(String clusterName, String resourceName) {
     HelixDataAccessor accessor =

http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
index 2820923..23b0df6 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
@@ -213,6 +213,16 @@ public class MockHelixAdmin implements HelixAdmin {
   }
 
   @Override
+  public void updateIdealState(String clusterName, String resourceName, IdealState idealState) {
+
+  }
+
+  @Override
+  public void removeFromIdealState(String clusterName, String resourceName, IdealState idealState) {
+
+  }
+
+  @Override
   public void enableInstance(String clusterName, String instanceName, boolean enabled) {
     String instanceConfigsPath = PropertyPathBuilder.instanceConfig(clusterName);
     if (!_baseDataAccessor.exists(instanceConfigsPath, 0)) {

http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index 9b8eabe..3b7d995 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -154,13 +154,12 @@ public class AbstractResource {
 
   protected Command getCommand(String commandStr) throws HelixException {
     if (commandStr == null) {
-      throw new HelixException("Unknown command " + commandStr);
+      throw new HelixException("Command string is null!");
     }
     try {
-      Command command = Command.valueOf(commandStr);
-      return command;
+      return Command.valueOf(commandStr);
     } catch (IllegalArgumentException ex) {
-      throw new HelixException("Unknown command " + commandStr);
+      throw new HelixException("Unknown command: " + commandStr);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
index 75d561b..ef9e096 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
@@ -355,6 +355,54 @@ public class ResourceAccessor extends AbstractHelixResource {
     return notFound();
   }
 
+  @POST
+  @Path("{resourceName}/idealState")
+  public Response updateResourceIdealState(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName, @QueryParam("command") String commandStr,
+      String content) {
+    Command command;
+    if (commandStr == null || commandStr.isEmpty()) {
+      command = Command.update; // Default behavior is update
+    } else {
+      try {
+        command = getCommand(commandStr);
+      } catch (HelixException ex) {
+        return badRequest(ex.getMessage());
+      }
+    }
+
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a vaild ZNRecord!");
+    }
+    IdealState idealState = new IdealState(record);
+    HelixAdmin helixAdmin = getHelixAdmin();
+    try {
+      switch (command) {
+      case update:
+        helixAdmin.updateIdealState(clusterId, resourceName, idealState);
+        break;
+      case delete: {
+        helixAdmin.removeFromIdealState(clusterId, resourceName, idealState);
+      }
+        break;
+      default:
+        return badRequest(String.format("Unsupported command: %s", command));
+      }
+    } catch (HelixException ex) {
+      return notFound(ex.getMessage()); // HelixAdmin throws a HelixException if it doesn't
+                                        // exist already
+    } catch (Exception ex) {
+      _logger.error(String.format("Failed to update the IdealState for resource: %s", resourceName),
+          ex);
+      return serverError(ex);
+    }
+    return OK();
+  }
+
   @GET
   @Path("{resourceName}/externalView")
   public Response getResourceExternalView(@PathParam("clusterId") String clusterId,

http://git-wip-us.apache.org/repos/asf/helix/blob/abc6969d/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
index db5c902..5aea447 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
@@ -32,10 +32,12 @@ import java.util.Set;
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import org.apache.helix.AccessOption;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
+import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.ExternalView;
@@ -343,9 +345,93 @@ public class TestResourceAccessor extends AbstractTestClass {
     }
   }
 
+
+  /**
+   * Test "update" command of updateResourceIdealState.
+   * @throws Exception
+   */
+  @Test(dependsOnMethods = "testResourceHealth")
+  public void updateResourceIdealState() throws Exception {
+    // Get IdealState ZNode
+    String zkPath = PropertyPathBuilder.idealState(CLUSTER_NAME, RESOURCE_NAME);
+    ZNRecord record = _baseAccessor.get(zkPath,  null, AccessOption.PERSISTENT);
+
+    // 1. Add these fields by way of "update"
+    Entity entity =
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/idealState",
+        Collections.singletonMap("command", "update"), entity, Response.Status.OK.getStatusCode());
+
+    // Check that the fields have been added
+    ZNRecord newRecord = _baseAccessor.get(zkPath, null, AccessOption.PERSISTENT);
+    Assert.assertEquals(record.getSimpleFields(), newRecord.getSimpleFields());
+    Assert.assertEquals(record.getListFields(), newRecord.getListFields());
+    Assert.assertEquals(record.getMapFields(), newRecord.getMapFields());
+
+    String newValue = "newValue";
+    // 2. Modify the record and update
+    for (int i = 0; i < 3; i++) {
+      String key = "k" + i;
+      record.getSimpleFields().put(key, newValue);
+      record.getMapFields().put(key, ImmutableMap.of(key, newValue));
+      record.getListFields().put(key, Arrays.asList(key, newValue));
+    }
+
+    entity =
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/idealState",
+        Collections.singletonMap("command", "update"), entity, Response.Status.OK.getStatusCode());
+
+    // Check that the fields have been modified
+    newRecord = _baseAccessor.get(zkPath, null, AccessOption.PERSISTENT);
+    Assert.assertEquals(record.getSimpleFields(), newRecord.getSimpleFields());
+    Assert.assertEquals(record.getListFields(), newRecord.getListFields());
+    Assert.assertEquals(record.getMapFields(), newRecord.getMapFields());
+  }
+
+  /**
+   * Test "delete" command of updateResourceIdealState.
+   * @throws Exception
+   */
+  @Test(dependsOnMethods = "updateResourceIdealState")
+  public void deleteFromResourceIdealState() throws Exception {
+    String zkPath = PropertyPathBuilder.idealState(CLUSTER_NAME, RESOURCE_NAME);
+    ZNRecord record = new ZNRecord(RESOURCE_NAME);
+
+    // Generate a record containing three keys (k1, k2, k3) for all fields for deletion
+    String value = "value";
+    for (int i = 1; i < 4; i++) {
+      String key = "k" + i;
+      record.getSimpleFields().put(key, value);
+      record.getMapFields().put(key, ImmutableMap.of(key, value));
+      record.getListFields().put(key, Arrays.asList(key, value));
+    }
+
+    // First, add these fields by way of "update"
+    Entity entity =
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/idealState",
+        Collections.singletonMap("command", "delete"), entity, Response.Status.OK.getStatusCode());
+
+    ZNRecord recordAfterDelete = _baseAccessor.get(zkPath, null, AccessOption.PERSISTENT);
+
+    // Check that the keys k1 and k2 have been deleted, and k0 remains
+    for (int i = 0; i < 4; i++) {
+      String key = "k" + i;
+      if (i == 0) {
+        Assert.assertTrue(recordAfterDelete.getSimpleFields().containsKey(key));
+        Assert.assertTrue(recordAfterDelete.getListFields().containsKey(key));
+        Assert.assertTrue(recordAfterDelete.getMapFields().containsKey(key));
+        continue;
+      }
+      Assert.assertFalse(recordAfterDelete.getSimpleFields().containsKey(key));
+      Assert.assertFalse(recordAfterDelete.getListFields().containsKey(key));
+      Assert.assertFalse(recordAfterDelete.getMapFields().containsKey(key));
+    }
+  }
+
   /**
    * Creates a setup where the health API can be tested.
-   *
    * @param clusterName
    * @param resourceName
    * @param idealStateParams