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/02 01:16:01 UTC

helix git commit: [HELIX-780] add task user content related api and added more tests

Repository: helix
Updated Branches:
  refs/heads/master 73c3f0ad8 -> 18aa67b6d


[HELIX-780] add task user content related api and added more tests


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

Branch: refs/heads/master
Commit: 18aa67b6d5c703e5b938b2f915f52a6ca856e889
Parents: 73c3f0a
Author: Harry Zhang <hr...@linkedin.com>
Authored: Tue Oct 9 14:31:00 2018 -0700
Committer: Harry Zhang <hr...@linkedin.com>
Committed: Thu Nov 1 17:42:22 2018 -0700

----------------------------------------------------------------------
 .../server/resources/helix/JobAccessor.java     |  11 +-
 .../server/resources/helix/TaskAccessor.java    | 115 +++++++++++++++++++
 .../resources/helix/WorkflowAccessor.java       |  10 +-
 .../helix/rest/server/AbstractTestClass.java    |  38 +++---
 .../helix/rest/server/TestJobAccessor.java      |  23 +++-
 .../helix/rest/server/TestTaskAccessor.java     |  83 +++++++++++++
 .../helix/rest/server/TestWorkflowAccessor.java |  23 +++-
 7 files changed, 278 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
index a984428..5548bd3 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
@@ -181,7 +181,9 @@ public class JobAccessor extends AbstractHelixResource {
       Map<String, String> contentStore =
           taskDriver.getJobUserContentMap(workflowName, jobName);
       if (contentStore == null) {
-        return JSONRepresentation(Collections.emptyMap());
+        return notFound(String.format(
+            "Unable to find content store. Workflow (%s) or Job (%s) does not exist.",
+            workflowName, jobName));
       }
       return JSONRepresentation(contentStore);
     } catch (ZkNoNodeException e) {
@@ -193,7 +195,7 @@ public class JobAccessor extends AbstractHelixResource {
 
   @POST
   @Path("{jobName}/userContent")
-  public Response updateWorkflowUserContent(
+  public Response updateJobUserContent(
       @PathParam("clusterId") String clusterId,
       @PathParam("workflowName") String workflowName,
       @PathParam("jobName") String jobName,
@@ -225,6 +227,11 @@ public class JobAccessor extends AbstractHelixResource {
       default:
         return badRequest(String.format("Command \"%s\" is not supported!", cmd));
       }
+    } catch (NullPointerException npe) {
+      // ZkCacheBasedDataAccessor would throw npe if workflow or job does not exist
+      return notFound(String.format(
+          "Unable to find content store. Workflow (%s) or Job (%s) does not exist.",
+          workflowName, jobName));
     } catch (Exception e) {
       _logger.error("Failed to update user content store", e);
       return serverError(e);

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/TaskAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/TaskAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/TaskAccessor.java
new file mode 100644
index 0000000..79290e1
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/TaskAccessor.java
@@ -0,0 +1,115 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * 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.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+import org.I0Itec.zkclient.exception.ZkNoNodeException;
+import org.apache.helix.task.TaskDriver;
+import org.codehaus.jackson.type.TypeReference;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Path("/clusters/{clusterId}/workflows/{workflowName}/jobs/{jobName}/tasks")
+public class TaskAccessor extends AbstractHelixResource {
+  private static Logger _logger = LoggerFactory.getLogger(TaskAccessor.class.getName());
+
+  @GET
+  @Path("{taskPartitionId}/userContent")
+  public Response getTaskUserContent(
+      @PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName,
+      @PathParam("jobName") String jobName,
+      @PathParam("taskPartitionId") String taskPartitionid
+  ) {
+    TaskDriver taskDriver = getTaskDriver(clusterId);
+    try {
+      Map<String, String> contentStore =
+          taskDriver.getTaskUserContentMap(workflowName, jobName, taskPartitionid);
+      if (contentStore == null) {
+        return notFound(String.format(
+            "Unable to find content store. Workflow (%s) or Job (%s) or Task content store (%s) not created yet.",
+            workflowName, jobName, taskPartitionid));
+      }
+      return JSONRepresentation(contentStore);
+    } catch (ZkNoNodeException e) {
+      return notFound(String.format(
+          "Unable to find content store. Workflow (%s) or Job (%s) not created yet.",
+          workflowName, jobName));
+    } catch (Exception e) {
+      return serverError(e);
+    }
+  }
+
+  @POST
+  @Path("{taskPartitionId}/userContent")
+  public Response updateTaskUserContent(
+      @PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName,
+      @PathParam("jobName") String jobName,
+      @PathParam("taskPartitionId") String taskPartitionid,
+      @QueryParam("command") String commandStr,
+      String content
+  ) {
+    Command cmd;
+    Map<String, String> contentMap = Collections.emptyMap();
+    try {
+      contentMap = OBJECT_MAPPER.readValue(content, new TypeReference<Map<String, String>>() {
+      });
+    } catch (IOException e) {
+      return badRequest(String
+          .format("Content %s cannot be deserialized to Map<String, String>. Err: %s", content,
+              e.getMessage()));
+    }
+
+    try {
+      cmd = (commandStr == null || commandStr.isEmpty())
+          ? Command.update
+          : Command.valueOf(commandStr);
+    } catch (IllegalArgumentException ie) {
+      return badRequest(String.format("Invalid command: %s. Err: %s", commandStr, ie.getMessage()));
+    }
+
+    TaskDriver driver = getTaskDriver(clusterId);
+    try {
+      switch (cmd) {
+      case update:
+        driver.addOrUpdateTaskUserContentMap(workflowName, jobName, taskPartitionid, contentMap);
+        return OK();
+      default:
+        return badRequest(String.format("Command \"%s\" is not supported!", cmd));
+      }
+    } catch (NullPointerException npe) {
+      // ZkCacheBasedDataAccessor would throw npe if workflow or job does not exist
+      return notFound(
+          String.format("Workflow (%s) or job (%s) does not exist", workflowName, jobName));
+    } catch (Exception e) {
+      _logger.error("Failed to update user content store", e);
+      return serverError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
index ac6a53c..8b328a7 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
@@ -276,7 +276,10 @@ public class WorkflowAccessor extends AbstractHelixResource {
       Map<String, String> contentStore =
           taskDriver.getWorkflowUserContentMap(workflowId);
       if (contentStore == null) {
-        return JSONRepresentation(Collections.emptyMap());
+        // ZkCacheBasedDataAccessor would throw npe if workflow or job does not exist
+        return notFound(String.format(
+            "Unable to find content store. Workflow (%s) does not exist.",
+            workflowId));
       }
       return JSONRepresentation(contentStore);
     } catch (ZkNoNodeException e) {
@@ -316,6 +319,11 @@ public class WorkflowAccessor extends AbstractHelixResource {
       default:
         return badRequest(String.format("Command \"%s\" is not supported!", cmd));
       }
+    } catch (NullPointerException npe) {
+      // ZkCacheBasedDataAccessor would throw npe if workflow or job does not exist
+      return notFound(String.format(
+          "Unable to find content store. Workflow (%s) does not exist.",
+          workflowId));
     } catch (Exception e) {
       _logger.error("Failed to update user content store", e);
       return serverError(e);

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
index f59db1a..97b735e 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
@@ -19,6 +19,21 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.Application;
+import javax.ws.rs.core.Response;
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
@@ -66,20 +81,6 @@ import org.testng.annotations.AfterSuite;
 import org.testng.annotations.BeforeSuite;
 
 import com.google.common.base.Joiner;
-import javax.ws.rs.client.Entity;
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.Application;
-import javax.ws.rs.core.Response;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Level;
 
 public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
   protected static final String ZK_ADDR = "localhost:2123";
@@ -389,10 +390,13 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
       _configAccessor.setResourceConfig(cluster, workflowName + "_" + JOB_PREFIX + i, job.build());
 
       // add job content stores
+      ZNRecord contentStore = new ZNRecord(TaskUtil.USER_CONTENT_NODE);
+      contentStore.setMapField(TaskUtil
+              .getNamespacedTaskName(TaskUtil.getNamespacedJobName(workflowName, JOB_PREFIX + i), "0"),
+          Collections.<String, String>emptyMap());
       propertyStore.create(Joiner.on("/")
-              .join(TaskConstants.REBALANCER_CONTEXT_ROOT, workflowName + "_" + JOB_PREFIX + i,
-                  TaskUtil.USER_CONTENT_NODE), new ZNRecord(TaskUtil.USER_CONTENT_NODE),
-          AccessOption.PERSISTENT);
+          .join(TaskConstants.REBALANCER_CONTEXT_ROOT, workflowName + "_" + JOB_PREFIX + i,
+              TaskUtil.USER_CONTENT_NODE), contentStore, AccessOption.PERSISTENT);
     }
     return jobCfgs;
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
index 1cf377b..d1178ee 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
@@ -153,7 +153,6 @@ public class TestJobAccessor extends AbstractTestClass {
     Map<String, String> map1 = new HashMap<>();
     map1.put("k1", "v1");
     Entity entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(map1), MediaType.APPLICATION_JSON_TYPE);
-    post(uri, ImmutableMap.of("command", "delete"), entity, Response.Status.BAD_REQUEST.getStatusCode());
     post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
 
     // update (add items) workflow content store
@@ -169,10 +168,30 @@ public class TestJobAccessor extends AbstractTestClass {
     body = get(uri, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertEquals(contentStore, map1);
+  }
 
+  @Test(dependsOnMethods = "testGetAddJobContent")
+  public void testInvalidGetAndUpdateJobContentStore() {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String validURI = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/Job_0/userContent";
+    String invalidURI1 = "clusters/" + CLUSTER_NAME + "/workflows/xxx/jobs/Job_0/userContent"; // workflow not exist
+    String invalidURI2 = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/xxx/userContent"; // job not exist
+    Entity validEntity = Entity.entity("{\"k1\":\"v1\"}", MediaType.APPLICATION_JSON_TYPE);
+    Entity invalidEntity = Entity.entity("{\"k1\":{}}", MediaType.APPLICATION_JSON_TYPE); // not Map<String, String>
+    Map<String, String> validCmd = ImmutableMap.of("command", "update");
+    Map<String, String> invalidCmd = ImmutableMap.of("command", "delete"); // cmd not supported
+
+    get(invalidURI1, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI2, Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    post(invalidURI1, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
+    post(invalidURI2, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
+
+    post(validURI, invalidCmd, validEntity, Response.Status.BAD_REQUEST.getStatusCode());
+    post(validURI, validCmd, invalidEntity, Response.Status.BAD_REQUEST.getStatusCode());
   }
 
-  @Test(dependsOnMethods = "testCreateJob")
+  @Test(dependsOnMethods = "testInvalidGetAndUpdateJobContentStore")
   public void testDeleteJob() {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
     TaskDriver driver = getTaskDriver(CLUSTER_NAME);

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/test/java/org/apache/helix/rest/server/TestTaskAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestTaskAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestTaskAccessor.java
new file mode 100644
index 0000000..575eae3
--- /dev/null
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestTaskAccessor.java
@@ -0,0 +1,83 @@
+package org.apache.helix.rest.server;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.helix.TestHelper;
+import org.codehaus.jackson.type.TypeReference;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestTaskAccessor extends AbstractTestClass {
+  private final static String CLUSTER_NAME = "TestCluster_0";
+
+  @Test
+  public void testGetAddTaskUserContent() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String uri = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/Job_0/tasks/0/userContent";
+    String uriTaskDoesNotExist = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/Job_0/tasks/xxx/userContent";
+
+    // Empty user content
+    String body =
+        get(uri, Response.Status.OK.getStatusCode(), true);
+    Map<String, String>
+        contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
+    Assert.assertTrue(contentStore.isEmpty());
+
+    // Post user content
+    Map<String, String> map1 = new HashMap<>();
+    map1.put("k1", "v1");
+    Entity entity =
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(map1), MediaType.APPLICATION_JSON_TYPE);
+    post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
+    post(uriTaskDoesNotExist, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
+
+    // get after post should work
+    body = get(uri, Response.Status.OK.getStatusCode(), true);
+    contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {
+    });
+    Assert.assertEquals(contentStore, map1);
+    body = get(uriTaskDoesNotExist, Response.Status.OK.getStatusCode(), true);
+    contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {
+    });
+    Assert.assertEquals(contentStore, map1);
+
+
+    // modify map1 and verify
+    map1.put("k1", "v2");
+    map1.put("k2", "v2");
+    entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(map1), MediaType.APPLICATION_JSON_TYPE);
+    post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
+    body = get(uri, Response.Status.OK.getStatusCode(), true);
+    contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {
+    });
+    Assert.assertEquals(contentStore, map1);
+  }
+
+  @Test
+  public void testInvalidGetAddTaskUserContent() {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String validURI = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/Job_0/tasks/0/userContent";
+    String invalidURI1 = "clusters/" + CLUSTER_NAME + "/workflows/xxx/jobs/Job_0/tasks/0/userContent"; // workflow not exist
+    String invalidURI2 = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/xxx/tasks/0/userContent"; // job not exist
+    String invalidURI3 = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/jobs/xxx/tasks/xxx/userContent"; // task not exist
+    Entity validEntity = Entity.entity("{\"k1\":\"v1\"}", MediaType.APPLICATION_JSON_TYPE);
+    Entity invalidEntity = Entity.entity("{\"k1\":{}}", MediaType.APPLICATION_JSON_TYPE); // not Map<String, String>
+    Map<String, String> validCmd = ImmutableMap.of("command", "update");
+    Map<String, String> invalidCmd = ImmutableMap.of("command", "delete"); // cmd not supported
+
+    get(invalidURI1, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI2, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI3, Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    post(invalidURI1, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
+    post(invalidURI2, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
+
+    post(validURI, invalidCmd, validEntity, Response.Status.BAD_REQUEST.getStatusCode());
+    post(validURI, validCmd, invalidEntity, Response.Status.BAD_REQUEST.getStatusCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/18aa67b6/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
index d622066..0f88841 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
@@ -18,6 +18,7 @@ import org.apache.helix.task.TaskState;
 import org.apache.helix.task.WorkflowConfig;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.type.TypeReference;
+import org.glassfish.jersey.server.spi.ResponseErrorMapper;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -140,7 +141,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
         TargetState.START);
   }
 
-  @Test(dependsOnMethods = "testCreateWorkflow")
+  @Test(dependsOnMethods = "testUpdateWorkflow")
   public void testGetAndUpdateWorkflowContentStore() throws IOException, InterruptedException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
     String workflowName = "Workflow_0";
@@ -157,7 +158,6 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     Map<String, String> map1 = new HashMap<>();
     map1.put("k1", "v1");
     Entity entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(map1), MediaType.APPLICATION_JSON_TYPE);
-    post(uri, ImmutableMap.of("command", "delete"), entity, Response.Status.BAD_REQUEST.getStatusCode());
     post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
 
     // update (add items) workflow content store
@@ -175,7 +175,24 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     Assert.assertEquals(contentStore, map1);
   }
 
-  @Test(dependsOnMethods = "testUpdateWorkflow")
+  @Test(dependsOnMethods = "testGetAndUpdateWorkflowContentStore")
+  public void testInvalidGetAndUpdateWorkflowContentStore() {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String validURI = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/userContent";
+    String invalidURI = "clusters/" + CLUSTER_NAME + "/workflows/xxx/userContent"; // workflow not exist
+    Entity validEntity = Entity.entity("{\"k1\":\"v1\"}", MediaType.APPLICATION_JSON_TYPE);
+    Entity invalidEntity = Entity.entity("{\"k1\":{}}", MediaType.APPLICATION_JSON_TYPE); // not Map<String, String>
+    Map<String, String> validCmd = ImmutableMap.of("command", "update");
+    Map<String, String> invalidCmd = ImmutableMap.of("command", "delete"); // cmd not supported
+
+    get(invalidURI, Response.Status.NOT_FOUND.getStatusCode(), false);
+    post(invalidURI, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
+
+    post(validURI, invalidCmd, validEntity, Response.Status.BAD_REQUEST.getStatusCode());
+    post(validURI, validCmd, invalidEntity, Response.Status.BAD_REQUEST.getStatusCode());
+  }
+
+  @Test(dependsOnMethods = "testInvalidGetAndUpdateWorkflowContentStore")
   public void testDeleteWorkflow() throws InterruptedException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
     TaskDriver driver = getTaskDriver(CLUSTER_NAME);