You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by hu...@apache.org on 2019/05/22 23:41:31 UTC

[helix] 06/14: get instance's pending messages with state model def parameter

This is an automated email from the ASF dual-hosted git repository.

hulee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/helix.git

commit babc471bbd81907da73f5f33b2f9f5a035e20177
Author: ywang4 <yw...@linkedin.com>
AuthorDate: Wed Feb 20 14:23:08 2019 -0800

    get instance's pending messages with state model def parameter
    
    Update the get() method in AbstractTestClass in order to take the correct QueryParam
    BUGS=HELIX-1645
    
    RB=1570393
    BUG=HELIX-1645
    G=helix-reviewers
    A=hulee,jxue
    
    Signed-off-by: Hunter Lee <hu...@linkedin.com>
---
 .../server/resources/helix/InstanceAccessor.java   | 29 +++++---
 .../helix/rest/server/AbstractTestClass.java       | 12 +++-
 .../helix/rest/server/TestClusterAccessor.java     | 18 ++---
 .../helix/rest/server/TestInstanceAccessor.java    | 77 ++++++++++++++++++++--
 .../apache/helix/rest/server/TestJobAccessor.java  | 22 ++++---
 .../helix/rest/server/TestNamespacedAPIAccess.java | 27 ++++----
 .../helix/rest/server/TestResourceAccessor.java    | 19 +++---
 .../apache/helix/rest/server/TestTaskAccessor.java | 14 ++--
 .../helix/rest/server/TestWorkflowAccessor.java    | 19 +++---
 9 files changed, 164 insertions(+), 73 deletions(-)

diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
index f55ee89..cfed0e2 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
 import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
@@ -47,12 +48,13 @@ import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ParticipantHistory;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.JsonNodeFactory;
 import org.codehaus.jackson.node.ObjectNode;
+import org.eclipse.jetty.util.StringUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Path("/clusters/{clusterId}/instances")
 public class InstanceAccessor extends AbstractHelixResource {
@@ -488,7 +490,8 @@ public class InstanceAccessor extends AbstractHelixResource {
   @GET
   @Path("{instanceName}/messages")
   public Response getMessagesOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
+      @PathParam("instanceName") String instanceName,
+      @QueryParam("stateModelDef") String stateModelDef) {
     HelixDataAccessor accessor = getDataAccssor(clusterId);
 
     ObjectNode root = JsonNodeFactory.instance.objectNode();
@@ -497,19 +500,27 @@ public class InstanceAccessor extends AbstractHelixResource {
     ArrayNode readMessages = root.putArray(InstanceProperties.read_messages.name());
 
 
-    List<String> messages =
+    List<String> messageNames =
         accessor.getChildNames(accessor.keyBuilder().messages(instanceName));
-    if (messages == null || messages.size() == 0) {
+    if (messageNames == null || messageNames.size() == 0) {
+      _logger.warn("Unable to get any messages on instance: " + instanceName);
       return notFound();
     }
 
-    for (String messageName : messages) {
+    for (String messageName : messageNames) {
       Message message = accessor.getProperty(accessor.keyBuilder().message(instanceName, messageName));
-      if (message.getMsgState() == Message.MessageState.NEW) {
-        newMessages.add(messageName);
+      if (message == null) {
+        _logger.warn("Message is deleted given message name: ", messageName);
+        continue;
+      }
+      // if stateModelDef is valid, keep messages with StateModelDef equals to the parameter
+      if (StringUtil.isNotBlank(stateModelDef) && !stateModelDef.equals(message.getStateModelDef())) {
+        continue;
       }
 
-      if (message.getMsgState() == Message.MessageState.READ) {
+      if (Message.MessageState.NEW.equals(message.getMsgState())) {
+        newMessages.add(messageName);
+      } else if (Message.MessageState.READ.equals(message.getMsgState())) {
         readMessages.add(messageName);
       }
     }
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 97b735e..01dd7b8 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
@@ -30,10 +30,12 @@ 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;
@@ -405,8 +407,14 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
     return OBJECT_MAPPER.reader(ZNRecord.class).readValue(data);
   }
 
-  protected String get(String uri, int expectedReturnStatus, boolean expectBodyReturned) {
-    final Response response = target(uri).request().get();
+  protected String get(String uri, Map<String, String> queryParams, int expectedReturnStatus, boolean expectBodyReturned) {
+    WebTarget webTarget = target(uri);
+    if (queryParams != null) {
+      for (Map.Entry<String, String> entry : queryParams.entrySet()) {
+        webTarget = webTarget.queryParam(entry.getKey(), entry.getValue());
+      }
+    }
+    final Response response = webTarget.request().get();
     Assert.assertEquals(response.getStatus(), expectedReturnStatus);
 
     // NOT_FOUND will throw text based html
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
index 3d7577a..59da6e1 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
@@ -69,7 +69,7 @@ public class TestClusterAccessor extends AbstractTestClass {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
     _auditLogger.clearupLogs();
-    String body = get("clusters", Response.Status.OK.getStatusCode(), true);
+    String body = get("clusters", null, Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String clustersStr = node.get(ClusterAccessor.ClusterProperties.clusters.name()).toString();
     Assert.assertNotNull(clustersStr);
@@ -255,14 +255,14 @@ public class TestClusterAccessor extends AbstractTestClass {
 
     // verify is in maintenance mode
     String body =
-        get("clusters/" + cluster + "/maintenance", Response.Status.OK.getStatusCode(), true);
+        get("clusters/" + cluster + "/maintenance", null, Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     boolean maintenance =
         node.get(ClusterAccessor.ClusterProperties.maintenance.name()).getBooleanValue();
     Assert.assertTrue(maintenance);
 
     // Check that we could retrieve maintenance signal correctly
-    String signal = get("clusters/" + cluster + "/controller/maintenanceSignal",
+    String signal = get("clusters/" + cluster + "/controller/maintenanceSignal", null,
         Response.Status.OK.getStatusCode(), true);
     Map<String, Object> maintenanceSignalMap =
         OBJECT_MAPPER.readValue(signal, new TypeReference<HashMap<String, Object>>() {
@@ -277,12 +277,12 @@ public class TestClusterAccessor extends AbstractTestClass {
         Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.OK.getStatusCode());
 
     // verify no longer in maintenance mode
-    body = get("clusters/" + cluster + "/maintenance", Response.Status.OK.getStatusCode(), true);
+    body = get("clusters/" + cluster + "/maintenance", null, Response.Status.OK.getStatusCode(), true);
     node = OBJECT_MAPPER.readTree(body);
     Assert.assertFalse(
         node.get(ClusterAccessor.ClusterProperties.maintenance.name()).getBooleanValue());
 
-    get("clusters/" + cluster + "/controller/maintenanceSignal",
+    get("clusters/" + cluster + "/controller/maintenanceSignal", null,
         Response.Status.NOT_FOUND.getStatusCode(), false);
   }
 
@@ -293,7 +293,7 @@ public class TestClusterAccessor extends AbstractTestClass {
 
     // Get the leader controller name for the cluster
     String leader =
-        get("clusters/" + cluster + "/controller", Response.Status.OK.getStatusCode(), true);
+        get("clusters/" + cluster + "/controller", null, Response.Status.OK.getStatusCode(), true);
     Map<String, String> leaderMap =
         OBJECT_MAPPER.readValue(leader, new TypeReference<HashMap<String, String>>() {
         });
@@ -302,7 +302,7 @@ public class TestClusterAccessor extends AbstractTestClass {
     Assert.assertNotNull(leader, "Leader name cannot be null!");
 
     // Get the controller leadership history JSON's last entry
-    String leadershipHistory = get("clusters/" + cluster + "/controller/history",
+    String leadershipHistory = get("clusters/" + cluster + "/controller/history", null,
         Response.Status.OK.getStatusCode(), true);
     Map<String, Object> leadershipHistoryMap =
         OBJECT_MAPPER.readValue(leadershipHistory, new TypeReference<HashMap<String, Object>>() {
@@ -330,7 +330,7 @@ public class TestClusterAccessor extends AbstractTestClass {
         Entity.entity(reason, MediaType.APPLICATION_JSON_TYPE), Response.Status.OK.getStatusCode());
 
     // Get the maintenance history JSON's last entry
-    String maintenanceHistory = get("clusters/" + cluster + "/controller/maintenanceHistory",
+    String maintenanceHistory = get("clusters/" + cluster + "/controller/maintenanceHistory", null,
         Response.Status.OK.getStatusCode(), true);
     Map<String, Object> maintenanceHistoryMap =
         OBJECT_MAPPER.readValue(maintenanceHistory, new TypeReference<HashMap<String, Object>>() {
@@ -372,7 +372,7 @@ public class TestClusterAccessor extends AbstractTestClass {
   }
 
   private ClusterConfig getClusterConfigFromRest(String cluster) throws IOException {
-    String body = get("clusters/" + cluster + "/configs", Response.Status.OK.getStatusCode(), true);
+    String body = get("clusters/" + cluster + "/configs", null, Response.Status.OK.getStatusCode(), true);
 
     ZNRecord record = new ObjectMapper().reader(ZNRecord.class).readValue(body);
     ClusterConfig clusterConfigRest = new ClusterConfig(record);
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 94c28b2..86c52e1 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -19,8 +19,6 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,28 +26,95 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 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.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.Message;
 import org.apache.helix.rest.server.resources.AbstractResource;
 import org.apache.helix.rest.server.resources.helix.InstanceAccessor;
 import org.codehaus.jackson.JsonNode;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
 public class TestInstanceAccessor extends AbstractTestClass {
   private final static String CLUSTER_NAME = "TestCluster_0";
   private final static String INSTANCE_NAME = CLUSTER_NAME + "localhost_12918";
 
   @Test
+  public void testGetAllMessages() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+
+    String messageId = "msg1";
+    Message message = new Message(Message.MessageType.STATE_TRANSITION, messageId);
+    message.setStateModelDef("MasterSlave");
+    message.setFromState("OFFLINE");
+    message.setToState("SLAVE");
+    message.setResourceName("testResourceName");
+    message.setPartitionName("testResourceName_1");
+    message.setTgtName("localhost_3");
+    message.setTgtSessionId("session_3");
+    HelixDataAccessor helixDataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+    helixDataAccessor.setProperty(helixDataAccessor.keyBuilder().message(INSTANCE_NAME, messageId), message);
+
+    String body =
+        get("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME + "/messages", null, Response.Status.OK.getStatusCode(), true);
+    JsonNode node = OBJECT_MAPPER.readTree(body);
+    int newMessageCount =
+        node.get(InstanceAccessor.InstanceProperties.total_message_count.name()).getIntValue();
+
+    Assert.assertEquals(newMessageCount, 1);
+  }
+
+  @Test (dependsOnMethods = "testGetAllMessages")
+  public void testGetMessagesByStateModelDef() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+
+    String messageId = "msg1";
+    Message message = new Message(Message.MessageType.STATE_TRANSITION, messageId);
+    message.setStateModelDef("MasterSlave");
+    message.setFromState("OFFLINE");
+    message.setToState("SLAVE");
+    message.setResourceName("testResourceName");
+    message.setPartitionName("testResourceName_1");
+    message.setTgtName("localhost_3");
+    message.setTgtSessionId("session_3");
+    HelixDataAccessor helixDataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+    helixDataAccessor.setProperty(helixDataAccessor.keyBuilder().message(INSTANCE_NAME, messageId), message);
+
+    String body = get("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME + "/messages",
+        ImmutableMap.of("stateModelDef", "MasterSlave"), Response.Status.OK.getStatusCode(), true);
+    JsonNode node = OBJECT_MAPPER.readTree(body);
+    int newMessageCount =
+        node.get(InstanceAccessor.InstanceProperties.total_message_count.name()).getIntValue();
+
+    Assert.assertEquals(newMessageCount, 1);
+
+    body = get("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME + "/messages",
+        ImmutableMap.of("stateModelDef", "LeaderStandBy"), Response.Status.OK.getStatusCode(),
+        true);
+    node = OBJECT_MAPPER.readTree(body);
+    newMessageCount =
+        node.get(InstanceAccessor.InstanceProperties.total_message_count.name()).getIntValue();
+
+    Assert.assertEquals(newMessageCount, 0);
+  }
+
+  @Test (dependsOnMethods = "testGetMessagesByStateModelDef")
   public void testGetInstances() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
     String body =
-        get("clusters/" + CLUSTER_NAME + "/instances", Response.Status.OK.getStatusCode(), true);
+        get("clusters/" + CLUSTER_NAME + "/instances", null, Response.Status.OK.getStatusCode(), true);
 
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String instancesStr =
@@ -66,7 +131,7 @@ public class TestInstanceAccessor extends AbstractTestClass {
   @Test(dependsOnMethods = "testGetInstances")
   public void testGetInstance() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
-    String body = get("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME,
+    String body = get("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME, null,
         Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String instancesCfg =
@@ -170,9 +235,7 @@ public class TestInstanceAccessor extends AbstractTestClass {
     // Test enable disable partitions
     String dbName = "_db_0_";
     List<String> partitionsToDisable = Arrays.asList(
-        new String[] { CLUSTER_NAME + dbName + "0", CLUSTER_NAME + dbName + "1",
-            CLUSTER_NAME + dbName + "3"
-        });
+         CLUSTER_NAME + dbName + "0", CLUSTER_NAME + dbName + "1", CLUSTER_NAME + dbName + "3");
 
     entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(ImmutableMap
             .of(AbstractResource.Properties.id.name(), INSTANCE_NAME,
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 d1178ee..82416fb 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
@@ -19,12 +19,12 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
-import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
@@ -43,6 +43,8 @@ import org.codehaus.jackson.type.TypeReference;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class TestJobAccessor extends AbstractTestClass {
   private final static String CLUSTER_NAME = "TestCluster_0";
   private final static String WORKFLOW_NAME = WORKFLOW_PREFIX + 0;
@@ -58,7 +60,7 @@ public class TestJobAccessor extends AbstractTestClass {
   public void testGetJobs() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs",
+    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs", null,
         Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String jobsStr = node.get(JobAccessor.JobProperties.Jobs.name()).toString();
@@ -74,7 +76,7 @@ public class TestJobAccessor extends AbstractTestClass {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
     String body =
-        get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs/" + JOB_NAME,
+        get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs/" + JOB_NAME, null,
             Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     Assert.assertNotNull(node.get(JobAccessor.JobProperties.JobConfig.name()));
@@ -91,7 +93,7 @@ public class TestJobAccessor extends AbstractTestClass {
 
     String body =
         get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs/" + JOB_NAME
-            + "/configs", Response.Status.OK.getStatusCode(), true);
+            + "/configs", null, Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String workflowId = node.get("simpleFields").get("WorkflowID").getTextValue();
     Assert.assertEquals(workflowId, WORKFLOW_NAME);
@@ -104,7 +106,7 @@ public class TestJobAccessor extends AbstractTestClass {
 
     String body =
         get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/jobs/" + JOB_NAME
-            + "/context", Response.Status.OK.getStatusCode(), true);
+            + "/context", null, Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     Assert.assertEquals(node.get("mapFields").get("0").get("STATE").getTextValue(),
         TaskPartitionState.COMPLETED.name());
@@ -145,7 +147,7 @@ public class TestJobAccessor extends AbstractTestClass {
 
     // Empty user content
     String body =
-        get(uri, Response.Status.OK.getStatusCode(), true);
+        get(uri, null, Response.Status.OK.getStatusCode(), true);
     Map<String, String> contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertTrue(contentStore.isEmpty());
 
@@ -156,7 +158,7 @@ public class TestJobAccessor extends AbstractTestClass {
     post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
 
     // update (add items) workflow content store
-    body = get(uri, Response.Status.OK.getStatusCode(), true);
+    body = get(uri, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertEquals(contentStore, map1);
 
@@ -165,7 +167,7 @@ public class TestJobAccessor extends AbstractTestClass {
     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);
+    body = get(uri, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertEquals(contentStore, map1);
   }
@@ -181,8 +183,8 @@ public class TestJobAccessor extends AbstractTestClass {
     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(invalidURI1, null, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI2, null, Response.Status.NOT_FOUND.getStatusCode(), false);
 
     post(invalidURI1, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
     post(invalidURI2, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
index c77f939..9ebad96 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
@@ -19,21 +19,24 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
-import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 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.PropertyKey;
 import org.apache.helix.rest.common.HelixRestNamespace;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class TestNamespacedAPIAccess extends AbstractTestClass {
   ObjectMapper _mapper = new ObjectMapper();
 
@@ -46,11 +49,11 @@ public class TestNamespacedAPIAccess extends AbstractTestClass {
     // Assume other api end points will behave the same way
     put(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName1), null,
         Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
-    get(String.format("/clusters/%s", testClusterName1), Response.Status.OK.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName1), null, Response.Status.OK.getStatusCode(), false);
 
     put(String.format("/clusters/%s", testClusterName2), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
         Response.Status.CREATED.getStatusCode());
-    get(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName2),
+    get(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName2), null,
         Response.Status.OK.getStatusCode(), false);
   }
 
@@ -62,14 +65,14 @@ public class TestNamespacedAPIAccess extends AbstractTestClass {
     // Create cluster in test namespace and verify it's only appears in test namespace
     put(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName), null,
         Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
-    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName), null,
         Response.Status.OK.getStatusCode(), false);
-    get(String.format("/clusters/%s", testClusterName), Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), null, Response.Status.NOT_FOUND.getStatusCode(), false);
 
     // Create cluster with same name in different namespacces
     put(String.format("/clusters/%s", testClusterName), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
         Response.Status.CREATED.getStatusCode());
-    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), null, Response.Status.OK.getStatusCode(), false);
 
     // Modify cluster in default namespace
     post(String.format("/clusters/%s", testClusterName), ImmutableMap.of("command", "disable"),
@@ -83,21 +86,21 @@ public class TestNamespacedAPIAccess extends AbstractTestClass {
     // Verify that deleting cluster in one namespace will not affect the other
     delete(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
         Response.Status.OK.getStatusCode());
-    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName), null,
         Response.Status.NOT_FOUND.getStatusCode(), false);
-    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), null, Response.Status.OK.getStatusCode(), false);
   }
 
   @Test
   public void testNamespaceServer() throws IOException {
     // Default endpoints should not have any namespace information returned
-    get("/", Response.Status.NOT_FOUND.getStatusCode(), false);
+    get("/", null, Response.Status.NOT_FOUND.getStatusCode(), false);
 
     // Get invalid namespace should return not found
-    get("/namespaces/invalid-namespace", Response.Status.NOT_FOUND.getStatusCode(), false);
+    get("/namespaces/invalid-namespace", null, Response.Status.NOT_FOUND.getStatusCode(), false);
 
     // list namespace should return a list of all namespaces
-    String body = get("/namespaces", Response.Status.OK.getStatusCode(), true);
+    String body = get("/namespaces", null, Response.Status.OK.getStatusCode(), true);
     List<Map<String, String>> namespaceMaps = _mapper
         .readValue(body, _mapper.getTypeFactory().constructCollectionType(List.class, Map.class));
     Assert.assertEquals(namespaceMaps.size(), 2);
@@ -126,7 +129,7 @@ public class TestNamespacedAPIAccess extends AbstractTestClass {
     Assert.assertTrue(expectedNamespaceNames.isEmpty());
 
     // Accessing root of namespaced API endpoint shall return information of that namespace
-    body = get(String.format("/namespaces/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME),
+    body = get(String.format("/namespaces/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME), null,
         Response.Status.OK.getStatusCode(), true);
     Map<String, String> namespace = _mapper.readValue(body,
         _mapper.getTypeFactory().constructMapType(Map.class, String.class, String.class));
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 5aea447..a90de7b 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
@@ -19,7 +19,6 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
-import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -29,9 +28,11 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 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;
@@ -50,6 +51,8 @@ import org.codehaus.jackson.type.TypeReference;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableMap;
+
 public class TestResourceAccessor extends AbstractTestClass {
   private final static String CLUSTER_NAME = "TestCluster_0";
   private final static String RESOURCE_NAME = CLUSTER_NAME + "_db_0";
@@ -60,7 +63,7 @@ public class TestResourceAccessor extends AbstractTestClass {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
     String body =
-        get("clusters/" + CLUSTER_NAME + "/resources", Response.Status.OK.getStatusCode(), true);
+        get("clusters/" + CLUSTER_NAME + "/resources", null, Response.Status.OK.getStatusCode(), true);
 
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String idealStates =
@@ -77,7 +80,7 @@ public class TestResourceAccessor extends AbstractTestClass {
   @Test(dependsOnMethods = "testGetResources")
   public void testGetResource() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
-    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME,
+    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME, null,
         Response.Status.OK.getStatusCode(), true);
 
     JsonNode node = OBJECT_MAPPER.readTree(body);
@@ -125,7 +128,7 @@ public class TestResourceAccessor extends AbstractTestClass {
   public void testResourceConfig() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/configs",
+    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/configs", null,
         Response.Status.OK.getStatusCode(), true);
     ResourceConfig resourceConfig = new ResourceConfig(toZNRecord(body));
     Assert.assertEquals(resourceConfig,
@@ -136,7 +139,7 @@ public class TestResourceAccessor extends AbstractTestClass {
   public void testIdealState() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/idealState",
+    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/idealState", null,
         Response.Status.OK.getStatusCode(), true);
     IdealState idealState = new IdealState(toZNRecord(body));
     Assert.assertEquals(idealState,
@@ -147,7 +150,7 @@ public class TestResourceAccessor extends AbstractTestClass {
   public void testExternalView() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/externalView",
+    String body = get("clusters/" + CLUSTER_NAME + "/resources/" + RESOURCE_NAME + "/externalView", null,
         Response.Status.OK.getStatusCode(), true);
     ExternalView externalView = new ExternalView(toZNRecord(body));
     Assert.assertEquals(externalView, _gSetupTool.getClusterManagementTool()
@@ -181,7 +184,7 @@ public class TestResourceAccessor extends AbstractTestClass {
     createDummyMapping(clusterName, resourceName, idealStateParams, partitionReplicaStates);
 
     // Get the result of getPartitionHealth
-    String body = get("clusters/" + clusterName + "/resources/" + resourceName + "/health",
+    String body = get("clusters/" + clusterName + "/resources/" + resourceName + "/health", null,
         Response.Status.OK.getStatusCode(), true);
 
     JsonNode node = OBJECT_MAPPER.readTree(body);
@@ -241,7 +244,7 @@ public class TestResourceAccessor extends AbstractTestClass {
     createDummyMapping(clusterName, resourceNameUnhealthy, idealStateParams, partitionReplicaStates_2);
 
     // Get the result of getResourceHealth
-    String body = get("clusters/" + clusterName + "/resources/health", Response.Status.OK.getStatusCode(),
+    String body = get("clusters/" + clusterName + "/resources/health", null, Response.Status.OK.getStatusCode(),
         true);
 
     JsonNode node = OBJECT_MAPPER.readTree(body);
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
index 575eae3..813f052 100644
--- 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
@@ -23,7 +23,7 @@ public class TestTaskAccessor extends AbstractTestClass {
 
     // Empty user content
     String body =
-        get(uri, Response.Status.OK.getStatusCode(), true);
+        get(uri, null, Response.Status.OK.getStatusCode(), true);
     Map<String, String>
         contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertTrue(contentStore.isEmpty());
@@ -37,11 +37,11 @@ public class TestTaskAccessor extends AbstractTestClass {
     post(uriTaskDoesNotExist, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
 
     // get after post should work
-    body = get(uri, Response.Status.OK.getStatusCode(), true);
+    body = get(uri, null, 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);
+    body = get(uriTaskDoesNotExist, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {
     });
     Assert.assertEquals(contentStore, map1);
@@ -52,7 +52,7 @@ public class TestTaskAccessor extends AbstractTestClass {
     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);
+    body = get(uri, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {
     });
     Assert.assertEquals(contentStore, map1);
@@ -70,9 +70,9 @@ public class TestTaskAccessor extends AbstractTestClass {
     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);
+    get(invalidURI1, null, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI2, null, Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(invalidURI3, null, Response.Status.NOT_FOUND.getStatusCode(), false);
 
     post(invalidURI1, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
     post(invalidURI2, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
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 0f88841..56a8853 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
@@ -5,9 +5,11 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 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.TestHelper;
 import org.apache.helix.rest.server.resources.helix.WorkflowAccessor;
 import org.apache.helix.task.JobQueue;
@@ -18,7 +20,6 @@ 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;
 
@@ -44,7 +45,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
     String body =
-        get("clusters/" + CLUSTER_NAME + "/workflows", Response.Status.OK.getStatusCode(), true);
+        get("clusters/" + CLUSTER_NAME + "/workflows", null, Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String workflowsStr = node.get(WorkflowAccessor.WorkflowProperties.Workflows.name()).toString();
     Set<String> workflows = OBJECT_MAPPER.readValue(workflowsStr,
@@ -55,7 +56,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
   @Test(dependsOnMethods = "testGetWorkflows")
   public void testGetWorkflow() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
-    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME,
+    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME, null,
         Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     Assert.assertNotNull(node.get(WorkflowAccessor.WorkflowProperties.WorkflowConfig.name()));
@@ -76,7 +77,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
   public void testGetWorkflowConfig() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/configs",
+    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/configs", null,
         Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     String workflowId = node.get("WorkflowID").getTextValue();
@@ -87,7 +88,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
   public void testGetWorkflowContext() throws IOException {
     System.out.println("Start test :" + TestHelper.getTestMethodName());
 
-    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/context",
+    String body = get("clusters/" + CLUSTER_NAME + "/workflows/" + WORKFLOW_NAME + "/context", null,
         Response.Status.OK.getStatusCode(), true);
     JsonNode node = OBJECT_MAPPER.readTree(body);
     Assert.assertEquals(node.get("STATE").getTextValue(),
@@ -151,7 +152,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     String uri = "clusters/" + CLUSTER_NAME + "/workflows/Workflow_0/userContent";
 
     String body =
-        get(uri, Response.Status.OK.getStatusCode(), true);
+        get(uri, null, Response.Status.OK.getStatusCode(), true);
     Map<String, String> contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertTrue(contentStore.isEmpty());
 
@@ -161,7 +162,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     post(uri, ImmutableMap.of("command", "update"), entity, Response.Status.OK.getStatusCode());
 
     // update (add items) workflow content store
-    body = get(uri, Response.Status.OK.getStatusCode(), true);
+    body = get(uri, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertEquals(contentStore, map1);
 
@@ -170,7 +171,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     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);
+    body = get(uri, null, Response.Status.OK.getStatusCode(), true);
     contentStore = OBJECT_MAPPER.readValue(body, new TypeReference<Map<String, String>>() {});
     Assert.assertEquals(contentStore, map1);
   }
@@ -185,7 +186,7 @@ public class TestWorkflowAccessor extends AbstractTestClass {
     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);
+    get(invalidURI, null, Response.Status.NOT_FOUND.getStatusCode(), false);
     post(invalidURI, validCmd, validEntity, Response.Status.NOT_FOUND.getStatusCode());
 
     post(validURI, invalidCmd, validEntity, Response.Status.BAD_REQUEST.getStatusCode());