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 2020/01/20 09:16:53 UTC

[helix] branch helix-cloud updated: Add REST API for Cluster Creation with CloudConfig (#675)

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

jxue pushed a commit to branch helix-cloud
in repository https://gitbox.apache.org/repos/asf/helix.git


The following commit(s) were added to refs/heads/helix-cloud by this push:
     new 2508d48  Add REST API for Cluster Creation with CloudConfig (#675)
2508d48 is described below

commit 2508d4853d558a3e786b4bf4eafd0135c6e37926
Author: Ali Reza Zamani Zadeh Najari <an...@linkedin.com>
AuthorDate: Mon Jan 20 01:16:45 2020 -0800

    Add REST API for Cluster Creation with CloudConfig (#675)
    
    This commit contains the relevant REST APIs for cluster creation.
    This change allows user to create cluster with CloudConfig.
    Multiple tests have been added in order to check the functionality of REST calls.
---
 .../server/resources/helix/ClusterAccessor.java    | 146 +++++++++-
 .../helix/rest/server/TestClusterAccessor.java     | 299 +++++++++++++++++++++
 2 files changed, 438 insertions(+), 7 deletions(-)

diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
index e4f0358..ae31cb5 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
@@ -47,6 +47,7 @@ import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.manager.zk.client.HelixZkClient;
+import org.apache.helix.model.CloudConfig;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.ControllerHistory;
 import org.apache.helix.model.HelixConfigScope;
@@ -132,17 +133,35 @@ public class ClusterAccessor extends AbstractHelixResource {
   @PUT
   @Path("{clusterId}")
   public Response createCluster(@PathParam("clusterId") String clusterId,
-      @DefaultValue("false") @QueryParam("recreate") String recreate) {
+      @DefaultValue("false") @QueryParam("recreate") String recreate,
+      @DefaultValue("false") @QueryParam("addCloudConfig") String addCloudConfig,
+      String content) {
+
     boolean recreateIfExists = Boolean.valueOf(recreate);
+    boolean cloudConfigIncluded = Boolean.valueOf(addCloudConfig);
+
+
     ClusterSetup clusterSetup = getClusterSetup();
 
-    try {
-      clusterSetup.addCluster(clusterId, recreateIfExists);
-    } catch (Exception ex) {
-      _logger.error("Failed to create cluster " + clusterId + ", exception: " + ex);
-      return serverError(ex);
-    }
+    CloudConfig cloudConfig = null;
 
+    if (cloudConfigIncluded) {
+      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!");
+      }
+      try {
+        cloudConfig = new CloudConfig.Builder(record).build();
+        clusterSetup.addCluster(clusterId, recreateIfExists, cloudConfig);
+      } catch (Exception ex) {
+        _logger.error("Error in adding a CloudConfig to cluster: " + clusterId, ex);
+        return badRequest(ex.getMessage());
+      }
+    }
+    clusterSetup.addCluster(clusterId, recreateIfExists, cloudConfig);
     return created();
   }
 
@@ -514,6 +533,119 @@ public class ClusterAccessor extends AbstractHelixResource {
     return ZKUtil.isClusterSetup(cluster, zkClient);
   }
 
+  @PUT
+  @Path("{clusterId}/cloudconfig")
+  public Response addCloudConfig(@PathParam("clusterId") String clusterId, String content) {
+
+    HelixZkClient zkClient = getHelixZkClient();
+    if (!ZKUtil.isClusterSetup(clusterId, zkClient)) {
+      return notFound("Cluster is not properly setup!");
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+    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!");
+    }
+
+    try {
+      CloudConfig cloudConfig = new CloudConfig.Builder(record).build();
+      admin.addCloudConfig(clusterId, cloudConfig);
+    } catch (HelixException ex) {
+      _logger.error("Error in adding a CloudConfig to cluster: " + clusterId, ex);
+      return badRequest(ex.getMessage());
+    } catch (Exception ex) {
+      _logger.error("Cannot add CloudConfig to cluster: " + clusterId, ex);
+      return serverError(ex);
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{clusterId}/cloudconfig")
+  public Response getCloudConfig(@PathParam("clusterId") String clusterId) {
+
+    HelixZkClient zkClient = getHelixZkClient();
+    if (!ZKUtil.isClusterSetup(clusterId, zkClient)) {
+      return notFound();
+    }
+
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    CloudConfig cloudConfig = configAccessor.getCloudConfig(clusterId);
+
+    if (cloudConfig != null) {
+      return JSONRepresentation(cloudConfig.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @POST
+  @Path("{clusterId}/cloudconfig")
+  public Response updateCloudConfig(@PathParam("clusterId") String clusterId,
+      @QueryParam("command") String commandStr, String content) {
+
+    HelixZkClient zkClient = getHelixZkClient();
+    if (!ZKUtil.isClusterSetup(clusterId, zkClient)) {
+      return notFound();
+    }
+
+    // Here to update cloud config
+    Command command;
+    if (commandStr == null || commandStr.isEmpty()) {
+      command = Command.update; // Default behavior
+    } else {
+      try {
+        command = getCommand(commandStr);
+      } catch (HelixException ex) {
+        return badRequest(ex.getMessage());
+      }
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+
+    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!");
+    }
+    try {
+      switch (command) {
+      case delete: {
+        admin.removeCloudConfig(clusterId);
+      }
+      break;
+      case update: {
+        try {
+          CloudConfig cloudConfig = new CloudConfig.Builder(record).build();
+          admin.removeCloudConfig(clusterId);
+          admin.addCloudConfig(clusterId, cloudConfig);
+        } catch (HelixException ex) {
+          _logger.error("Error in updating a CloudConfig to cluster: " + clusterId, ex);
+          return badRequest(ex.getMessage());
+        } catch (Exception ex) {
+          _logger.error("Cannot update CloudConfig for cluster: " + clusterId, ex);
+          return serverError(ex);
+        }
+      }
+      break;
+      default:
+        return badRequest("Unsupported command " + commandStr);
+      }
+    } catch (Exception ex) {
+      _logger.error("Failed to " + command + " cloud config, cluster " + clusterId + " new config: "
+          + content + ", Exception: " + ex);
+      return serverError(ex);
+    }
+    return OK();
+  }
+
   /**
    * Reads HISTORY ZNode from the metadata store and generates a Map object that contains the
    * pertinent history entries depending on the history type.
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 b8d8018..4ea2d79 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
@@ -33,6 +33,7 @@ import javax.ws.rs.core.Response;
 
 import com.google.common.collect.ImmutableMap;
 import com.sun.research.ws.wadl.HTTPMethods;
+import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
@@ -48,6 +49,8 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.CloudConfig;
+import org.apache.helix.cloud.constants.CloudProvider;
 import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.resources.AbstractResource;
@@ -563,6 +566,302 @@ public class TestClusterAccessor extends AbstractTestClass {
     System.out.println("End test :" + TestHelper.getTestMethodName());
   }
 
+
+  @Test(dependsOnMethods = "testActivateSuperCluster")
+  public void testAddClusterWithCloudConfig() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    ZNRecord record = new ZNRecord("testZnode");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.AZURE.name());
+
+    Map<String, String> map = new HashMap<>();
+    map.put("addCloudConfig", "true");
+
+    put("clusters/" + clusterName, map,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+
+    // Read CloudConfig from Zookeeper and check the content
+    ConfigAccessor _configAccessor = new ConfigAccessor(_gZkClient);
+    CloudConfig cloudConfigFromZk = _configAccessor.getCloudConfig(clusterName);
+    Assert.assertTrue(cloudConfigFromZk.isCloudEnabled());
+    Assert.assertEquals(cloudConfigFromZk.getCloudID(), "TestCloudID");
+    Assert.assertEquals(cloudConfigFromZk.getCloudProvider(), CloudProvider.AZURE.name());
+  }
+
+  @Test(dependsOnMethods = "testAddClusterWithCloudConfig")
+  public void testAddClusterWithInvalidCloudConfig() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    ZNRecord record = new ZNRecord("testZnode");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+
+    Map<String, String> map = new HashMap<>();
+    map.put("addCloudConfig", "true");
+
+    // Cloud Provider has not been defined. Result of this rest call will be BAD_REQUEST.
+    put("clusters/" + clusterName, map,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.BAD_REQUEST.getStatusCode());
+  }
+
+  @Test(dependsOnMethods = "testAddClusterWithInvalidCloudConfig")
+  public void testAddClusterWithInvalidCustomizedCloudConfig() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    ZNRecord record = new ZNRecord("testZnode");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.CUSTOMIZED.name());
+
+    Map<String, String> map = new HashMap<>();
+    map.put("addCloudConfig", "true");
+
+    // Cloud Provider is customized. CLOUD_INFO_PROCESSOR_NAME and CLOUD_INFO_SOURCE fields are
+    // required.
+    put("clusters/" + clusterName, map,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.BAD_REQUEST.getStatusCode());
+  }
+
+  @Test(dependsOnMethods = "testAddClusterWithInvalidCustomizedCloudConfig")
+  public void testAddClusterWithValidCustomizedCloudConfig() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    ZNRecord record = new ZNRecord("testZnode");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.CUSTOMIZED.name());
+    List<String> sourceList = new ArrayList<String>();
+    sourceList.add("TestURL");
+    record.setListField(CloudConfig.CloudConfigProperty.CLOUD_INFO_SOURCE.name(), sourceList);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_INFO_PROCESSOR_NAME.name(),
+        "TestProcessorName");
+
+    Map<String, String> map = new HashMap<>();
+    map.put("addCloudConfig", "true");
+
+    // Cloud Provider is customized. CLOUD_INFO_PROCESSOR_NAME and CLOUD_INFO_SOURCE fields are
+    // required.
+    put("clusters/" + clusterName, map,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+
+    // Read CloudConfig from Zookeeper and check the content
+    ConfigAccessor _configAccessor = new ConfigAccessor(_gZkClient);
+    CloudConfig cloudConfigFromZk = _configAccessor.getCloudConfig(clusterName);
+    Assert.assertTrue(cloudConfigFromZk.isCloudEnabled());
+    Assert.assertEquals(cloudConfigFromZk.getCloudID(), "TestCloudID");
+    List<String> listUrlFromZk = cloudConfigFromZk.getCloudInfoSources();
+    Assert.assertEquals(listUrlFromZk.get(0), "TestURL");
+    Assert.assertEquals(cloudConfigFromZk.getCloudInfoProcessorName(), "TestProcessorName");
+    Assert.assertEquals(cloudConfigFromZk.getCloudProvider(), CloudProvider.CUSTOMIZED.name());
+  }
+
+  @Test(dependsOnMethods = "testAddClusterWithValidCustomizedCloudConfig")
+  public void testAddClusterWithCloudConfigDisabledCloud() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    ZNRecord record = new ZNRecord("testZnode");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), false);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.AZURE.name());
+
+    Map<String, String> map = new HashMap<>();
+    map.put("addCloudConfig", "true");
+
+    put("clusters/" + clusterName, map,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+
+    // Read CloudConfig from Zookeeper and check the content
+    ConfigAccessor _configAccessor = new ConfigAccessor(_gZkClient);
+    CloudConfig cloudConfigFromZk = _configAccessor.getCloudConfig(clusterName);
+    Assert.assertFalse(cloudConfigFromZk.isCloudEnabled());
+    Assert.assertEquals(cloudConfigFromZk.getCloudID(), "TestCloudID");
+    Assert.assertEquals(cloudConfigFromZk.getCloudProvider(), CloudProvider.AZURE.name());
+  }
+
+
+  @Test(dependsOnMethods = "testAddClusterWithCloudConfigDisabledCloud")
+  public void testAddCloudConfigNonExistedCluster() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String urlBase = "clusters/TestCloud/cloudconfig/";
+    ZNRecord record = new ZNRecord("TestCloud");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.AZURE.name());
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    List<String> testList = new ArrayList<String>();
+    testList.add("TestURL");
+    record.setListField(CloudConfig.CloudConfigProperty.CLOUD_INFO_SOURCE.name(), testList);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_INFO_PROCESSOR_NAME.name(),
+        "TestProcessor");
+
+    // Not found since the cluster is not setup yet.
+    put(urlBase, null,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.NOT_FOUND.getStatusCode());
+    System.out.println("End test :" + TestHelper.getTestMethodName());
+  }
+
+  @Test(dependsOnMethods = "testAddCloudConfigNonExistedCluster")
+  public void testAddCloudConfig() throws Exception {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    _gSetupTool.addCluster("TestCloud", true);
+    String urlBase = "clusters/TestCloud/cloudconfig/";
+
+    ZNRecord record = new ZNRecord("TestCloud");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.CUSTOMIZED.name());
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    List<String> testList = new ArrayList<String>();
+    testList.add("TestURL");
+    record.setListField(CloudConfig.CloudConfigProperty.CLOUD_INFO_SOURCE.name(), testList);
+
+    // Bad request since Processor has not been defined.
+    put(urlBase, null,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.BAD_REQUEST.getStatusCode());
+
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_INFO_PROCESSOR_NAME.name(),
+        "TestProcessorName");
+
+    // Now response should be OK since all fields are set
+    put(urlBase, null,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+
+    // Read CloudConfig from Zookeeper and check the content
+    ConfigAccessor _configAccessor = new ConfigAccessor(_gZkClient);
+    CloudConfig cloudConfigFromZk = _configAccessor.getCloudConfig("TestCloud");
+    Assert.assertTrue(cloudConfigFromZk.isCloudEnabled());
+    Assert.assertEquals(cloudConfigFromZk.getCloudID(), "TestCloudID");
+    List<String> listUrlFromZk = cloudConfigFromZk.getCloudInfoSources();
+    Assert.assertEquals(listUrlFromZk.get(0), "TestURL");
+    Assert.assertEquals(cloudConfigFromZk.getCloudInfoProcessorName(), "TestProcessorName");
+    Assert.assertEquals(cloudConfigFromZk.getCloudProvider(), CloudProvider.CUSTOMIZED.name());
+
+    // Now test the getCloudConfig method.
+    String body = get(urlBase, null, Response.Status.OK.getStatusCode(), true);
+
+    ZNRecord recordFromRest = new ObjectMapper().reader(ZNRecord.class).readValue(body);
+    CloudConfig cloudConfigRest = new CloudConfig.Builder(recordFromRest).build();
+    CloudConfig cloudConfigZk = _configAccessor.getCloudConfig("TestCloud");
+
+    // Check that the CloudConfig from Zk and REST get method are equal
+    Assert.assertEquals(cloudConfigRest, cloudConfigZk);
+
+    // Check the fields individually
+    Assert.assertTrue(cloudConfigRest.isCloudEnabled());
+    Assert.assertEquals(cloudConfigRest.getCloudID(), "TestCloudID");
+    Assert.assertEquals(cloudConfigRest.getCloudProvider(), CloudProvider.CUSTOMIZED.name());
+    List<String> listUrlFromRest = cloudConfigRest.getCloudInfoSources();
+    Assert.assertEquals(listUrlFromRest.get(0), "TestURL");
+    Assert.assertEquals(cloudConfigRest.getCloudInfoProcessorName(), "TestProcessorName");
+
+    System.out.println("End test :" + TestHelper.getTestMethodName());
+  }
+
+  @Test(dependsOnMethods = "testAddCloudConfig")
+  public void testDeleteCloudConfig() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    _gSetupTool.addCluster("TestCloud", true);
+    String urlBase = "clusters/TestCloud/cloudconfig/";
+
+    Map<String, String> map1 = new HashMap<>();
+    map1.put("command", AbstractResource.Command.delete.name());
+
+    ZNRecord record = new ZNRecord("TestCloud");
+    post(urlBase, map1, Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+
+    // Read CloudConfig from Zookeeper and make sure it has been removed
+    ConfigAccessor _configAccessor = new ConfigAccessor(_gZkClient);
+    CloudConfig cloudConfigFromZk = _configAccessor.getCloudConfig("TestCloud");
+    Assert.assertNull(cloudConfigFromZk);
+
+    System.out.println("End test :" + TestHelper.getTestMethodName());
+  }
+
+  @Test(dependsOnMethods = "testDeleteCloudConfig")
+  public void testUpdateCloudConfig() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    _gSetupTool.addCluster("TestCloud", true);
+    String urlBase = "clusters/TestCloud/cloudconfig/";
+
+    ZNRecord record = new ZNRecord("TestCloud");
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudID");
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.AZURE.name());
+
+    // Fist add CloudConfig to the cluster
+    put(urlBase, null,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+
+    // Now get the Cloud Config and make sure the information is correct
+    String body = get(urlBase, null, Response.Status.OK.getStatusCode(), true);
+
+    ZNRecord recordFromRest = new ObjectMapper().reader(ZNRecord.class).readValue(body);
+    CloudConfig cloudConfigRest = new CloudConfig.Builder(recordFromRest).build();
+    Assert.assertTrue(cloudConfigRest.isCloudEnabled());
+    Assert.assertEquals(cloudConfigRest.getCloudID(), "TestCloudID");
+    Assert.assertEquals(cloudConfigRest.getCloudProvider(), CloudProvider.AZURE.name());
+
+    // Now put new information in the ZNRecord
+    record.setBooleanField(CloudConfig.CloudConfigProperty.CLOUD_ENABLED.name(), true);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_PROVIDER.name(),
+        CloudProvider.CUSTOMIZED.name());
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_ID.name(), "TestCloudIdNew");
+    List<String> testList = new ArrayList<String>();
+    testList.add("TestURL");
+    record.setListField(CloudConfig.CloudConfigProperty.CLOUD_INFO_SOURCE.name(), testList);
+    record.setSimpleField(CloudConfig.CloudConfigProperty.CLOUD_INFO_PROCESSOR_NAME.name(),
+        "TestProcessorName");
+
+    Map<String, String> map1 = new HashMap<>();
+    map1.put("command", AbstractResource.Command.update.name());
+
+    post(urlBase, map1,
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+
+    // Now get the Cloud Config and make sure the information has been updated
+    body = get(urlBase, null, Response.Status.OK.getStatusCode(), true);
+
+    recordFromRest = new ObjectMapper().reader(ZNRecord.class).readValue(body);
+    cloudConfigRest = new CloudConfig.Builder(recordFromRest).build();
+    Assert.assertTrue(cloudConfigRest.isCloudEnabled());
+    Assert.assertEquals(cloudConfigRest.getCloudID(), "TestCloudIdNew");
+    Assert.assertEquals(cloudConfigRest.getCloudProvider(), CloudProvider.CUSTOMIZED.name());
+    List<String> listUrlFromRest = cloudConfigRest.getCloudInfoSources();
+    Assert.assertEquals(listUrlFromRest.get(0), "TestURL");
+    Assert.assertEquals(cloudConfigRest.getCloudInfoProcessorName(), "TestProcessorName");
+
+    System.out.println("End test :" + TestHelper.getTestMethodName());
+  }
+
+
   private ClusterConfig getClusterConfigFromRest(String cluster) throws IOException {
     String body = get("clusters/" + cluster + "/configs", null, Response.Status.OK.getStatusCode(), true);