You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@heron.apache.org by GitBox <gi...@apache.org> on 2018/04/04 22:39:05 UTC

[GitHub] Yaliang closed pull request #2842: Add json format for dry-run

Yaliang closed pull request #2842: Add json format for dry-run
URL: https://github.com/apache/incubator-heron/pull/2842
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/heron/common/src/java/com/twitter/heron/common/basics/DryRunFormatType.java b/heron/common/src/java/com/twitter/heron/common/basics/DryRunFormatType.java
index ff85d4f157..407ac4c56d 100644
--- a/heron/common/src/java/com/twitter/heron/common/basics/DryRunFormatType.java
+++ b/heron/common/src/java/com/twitter/heron/common/basics/DryRunFormatType.java
@@ -20,7 +20,8 @@
 public enum DryRunFormatType {
   RAW,
   TABLE,
-  COLORED_TABLE;
+  COLORED_TABLE,
+  JSON;
 
   public static DryRunFormatType getDryRunFormatType(String dryRunFormatType) {
     return DryRunFormatType.valueOf(dryRunFormatType.toUpperCase());
diff --git a/heron/scheduler-core/src/java/BUILD b/heron/scheduler-core/src/java/BUILD
index 30aa30b4b4..b471af4818 100644
--- a/heron/scheduler-core/src/java/BUILD
+++ b/heron/scheduler-core/src/java/BUILD
@@ -1,13 +1,14 @@
 package(default_visibility = ["//visibility:public"])
 
 common_deps_files = [
-   "//heron/common/src/java:basics-java",
-   "//heron/common/src/java:config-java",
-   "//heron/common/src/java:utils-java",
-   "//heron/api/src/java:classification",
-   "@commons_cli_commons_cli//jar",
-   "@com_google_guava_guava//jar",
-   "@org_apache_commons_commons_lang3//jar",
+    "//heron/common/src/java:basics-java",
+    "//heron/common/src/java:config-java",
+    "//heron/common/src/java:utils-java",
+    "//heron/api/src/java:classification",
+    "@commons_cli_commons_cli//jar",
+    "@com_google_guava_guava//jar",
+    "@org_apache_commons_commons_lang3//jar",
+    "//third_party/java:jackson",
 ]
 
 spi_deps_files = [
diff --git a/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtils.java b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtils.java
new file mode 100644
index 0000000000..c2c0b6c98e
--- /dev/null
+++ b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtils.java
@@ -0,0 +1,87 @@
+//  Copyright 2018 Twitter. All rights reserved.
+//
+//  Licensed 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.
+package com.twitter.heron.scheduler.dryrun;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import com.twitter.heron.spi.packing.PackingPlan;
+import com.twitter.heron.spi.packing.Resource;
+
+public class JsonFormatterUtils {
+
+  private final ObjectMapper mapper;
+
+  public JsonFormatterUtils() {
+    mapper = new ObjectMapper();
+  }
+
+  public String renderPackingPlan(String topologyName, String packingClass,
+                                  PackingPlan packingPlan) throws JsonProcessingException {
+    ObjectNode topLevel = mapper.createObjectNode();
+    topLevel.put("packingClass", packingClass);
+    topLevel.put("topology", topologyName);
+
+    ArrayNode containers = mapper.createArrayNode();
+
+    for (PackingPlan.ContainerPlan containerPlan : packingPlan.getContainers()) {
+      ObjectNode planNode = mapper.createObjectNode();
+      planNode.put("taskId", containerPlan.getId());
+      containers.add(renderContainerPlan(containerPlan));
+    }
+
+    topLevel.set("containers", containers);
+
+    return mapper.writeValueAsString(topLevel);
+  }
+
+  ObjectNode renderContainerPlan(PackingPlan.ContainerPlan containerPlan) {
+    Resource requiredResources = containerPlan.getRequiredResource();
+    ObjectNode resources = mapper.createObjectNode();
+    resources.put("cpu", requiredResources.getCpu());
+    resources.put("ram", requiredResources.getRam().asBytes());
+    resources.put("disk", requiredResources.getDisk().asBytes());
+
+    ObjectNode containerNode = mapper.createObjectNode();
+    containerNode.put("id", containerPlan.getId());
+    containerNode.set("resources", resources);
+
+    ArrayNode components = mapper.createArrayNode();
+    for (PackingPlan.InstancePlan instancePlan : containerPlan.getInstances()) {
+      components.add(renderInstancePlan(instancePlan));
+    }
+
+    containerNode.set("components", components);
+
+    return containerNode;
+  }
+
+  ObjectNode renderInstancePlan(PackingPlan.InstancePlan instancePlan) {
+    Resource resources = instancePlan.getResource();
+    ObjectNode resourcesNode = mapper.createObjectNode();
+    resourcesNode.put("cpu", resources.getCpu());
+    resourcesNode.put("ram", resources.getRam().asBytes());
+    resourcesNode.put("disk", resources.getDisk().asBytes());
+
+    ObjectNode instancePlanNode = mapper.createObjectNode();
+    instancePlanNode.put("component", instancePlan.getComponentName());
+    instancePlanNode.put("id", instancePlan.getTaskId());
+    instancePlanNode.put("index", instancePlan.getComponentIndex());
+    instancePlanNode.set("resources", resourcesNode);
+
+    return instancePlanNode;
+  }
+}
diff --git a/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/SubmitJsonDryRunRenderer.java b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/SubmitJsonDryRunRenderer.java
new file mode 100644
index 0000000000..65cb6d64ad
--- /dev/null
+++ b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/SubmitJsonDryRunRenderer.java
@@ -0,0 +1,44 @@
+// Copyright 2016 Twitter. All rights reserved.
+//
+// Licensed 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.
+package com.twitter.heron.scheduler.dryrun;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import com.twitter.heron.spi.common.Context;
+import com.twitter.heron.spi.packing.PackingPlan;
+
+/**
+ * Dry-run renderer that renders submit dry-run response in raw format
+ */
+public class SubmitJsonDryRunRenderer {
+  private final SubmitDryRunResponse response;
+  private final JsonFormatterUtils formatter;
+
+  public SubmitJsonDryRunRenderer(SubmitDryRunResponse response) {
+    this.response = response;
+    this.formatter = new JsonFormatterUtils();
+  }
+
+  public String render() {
+    String topologyName = response.getTopology().getName();
+    String packingClassName = Context.packingClass(response.getConfig());
+    PackingPlan packingPlan = response.getPackingPlan();
+
+    try {
+      return formatter.renderPackingPlan(topologyName, packingClassName, packingPlan);
+    } catch (JsonProcessingException e) {
+      return "ERROR: " + e.getMessage();
+    }
+  }
+}
diff --git a/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/UpdateJsonDryRunRenderer.java b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/UpdateJsonDryRunRenderer.java
new file mode 100644
index 0000000000..05bc24b2dd
--- /dev/null
+++ b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/dryrun/UpdateJsonDryRunRenderer.java
@@ -0,0 +1,57 @@
+//  Copyright 2018 Twitter. All rights reserved.
+//
+//  Licensed 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.
+package com.twitter.heron.scheduler.dryrun;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import com.twitter.heron.spi.common.Context;
+import com.twitter.heron.spi.packing.PackingPlan;
+
+/**
+ * Dry-run renderer that renders submit dry-run response in raw format
+ */
+public class UpdateJsonDryRunRenderer {
+  private final UpdateDryRunResponse response;
+
+  public UpdateJsonDryRunRenderer(UpdateDryRunResponse response) {
+    this.response = response;
+  }
+
+  public String render() {
+    StringBuilder builder = new StringBuilder();
+    JsonFormatterUtils newFormatter = new JsonFormatterUtils();
+    JsonFormatterUtils oldFormatter = new JsonFormatterUtils();
+
+    String topologyName = response.getTopology().getName();
+    String packingClassName = Context.packingClass(response.getConfig());
+    PackingPlan newPackingPlan = response.getPackingPlan();
+    PackingPlan oldPackingPlan = response.getOldPackingPlan();
+
+    try {
+      String newPackingPlanJson = newFormatter.renderPackingPlan(topologyName, packingClassName,
+          newPackingPlan);
+      String oldPackingPlanJson = oldFormatter.renderPackingPlan(topologyName, packingClassName,
+          oldPackingPlan);
+
+      builder.append("New packing plan:\n");
+      builder.append(newPackingPlanJson).append("\n");
+      builder.append("Old packing plan:\n");
+      builder.append(oldPackingPlanJson).append("\n");
+
+      return builder.toString();
+    } catch (JsonProcessingException e) {
+      return "ERROR: " + e.getMessage();
+    }
+  }
+}
diff --git a/heron/scheduler-core/src/java/com/twitter/heron/scheduler/utils/DryRunRenders.java b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/utils/DryRunRenders.java
index d62fdfa67a..fde99f5bae 100644
--- a/heron/scheduler-core/src/java/com/twitter/heron/scheduler/utils/DryRunRenders.java
+++ b/heron/scheduler-core/src/java/com/twitter/heron/scheduler/utils/DryRunRenders.java
@@ -14,10 +14,13 @@
 package com.twitter.heron.scheduler.utils;
 
 import com.twitter.heron.common.basics.DryRunFormatType;
+
 import com.twitter.heron.scheduler.dryrun.SubmitDryRunResponse;
+import com.twitter.heron.scheduler.dryrun.SubmitJsonDryRunRenderer;
 import com.twitter.heron.scheduler.dryrun.SubmitRawDryRunRenderer;
 import com.twitter.heron.scheduler.dryrun.SubmitTableDryRunRenderer;
 import com.twitter.heron.scheduler.dryrun.UpdateDryRunResponse;
+import com.twitter.heron.scheduler.dryrun.UpdateJsonDryRunRenderer;
 import com.twitter.heron.scheduler.dryrun.UpdateRawDryRunRenderer;
 import com.twitter.heron.scheduler.dryrun.UpdateTableDryRunRenderer;
 
@@ -31,6 +34,8 @@ public static String render(SubmitDryRunResponse response, DryRunFormatType form
         return new SubmitTableDryRunRenderer(response, false).render();
       case COLORED_TABLE:
         return new SubmitTableDryRunRenderer(response, true).render();
+      case JSON:
+        return new SubmitJsonDryRunRenderer(response).render();
       default: throw new IllegalArgumentException(
           String.format("Unexpected rendering format: %s", formatType));
     }
@@ -44,6 +49,8 @@ public static String render(UpdateDryRunResponse response, DryRunFormatType form
         return new UpdateTableDryRunRenderer(response, false).render();
       case COLORED_TABLE:
         return new UpdateTableDryRunRenderer(response, true).render();
+      case JSON:
+        return new UpdateJsonDryRunRenderer(response).render();
       default: throw new IllegalArgumentException(
           String.format("Unexpected rendering format: %s", formatType));
     }
diff --git a/heron/scheduler-core/tests/java/BUILD b/heron/scheduler-core/tests/java/BUILD
index 1fd405a520..a4fc8e3bf6 100644
--- a/heron/scheduler-core/tests/java/BUILD
+++ b/heron/scheduler-core/tests/java/BUILD
@@ -11,47 +11,48 @@ common_deps_files = [
 ]
 
 spi_deps_files = [
-  "//heron/spi/src/java:common-spi-java",
-  "//heron/spi/src/java:statemgr-spi-java",
-  "//heron/spi/src/java:uploader-spi-java",
-  "//heron/spi/src/java:packing-spi-java",
-  "//heron/spi/src/java:scheduler-spi-java",
-  "//heron/spi/src/java:utils-spi-java",
+    "//heron/spi/src/java:common-spi-java",
+    "//heron/spi/src/java:statemgr-spi-java",
+    "//heron/spi/src/java:uploader-spi-java",
+    "//heron/spi/src/java:packing-spi-java",
+    "//heron/spi/src/java:scheduler-spi-java",
+    "//heron/spi/src/java:utils-spi-java",
 ]
 
 scheduler_deps_files = \
-  common_deps_files + \
-  heron_java_proto_files() + \
-  spi_deps_files
+    common_deps_files + \
+    heron_java_proto_files() + \
+    spi_deps_files
 
 java_library(
-  name = "scheduler-core-tests",
-  srcs = glob(["**/*.java"]),
-  deps = scheduler_deps_files + [
-    "//heron/statemgrs/src/java:null-statemgr-java",
-    "//heron/schedulers/src/java:null-scheduler-java",
-    "//heron/packing/src/java:roundrobin-packing",
-  ],
+    name = "scheduler-core-tests",
+    srcs = glob(["**/*.java"]),
+    deps = scheduler_deps_files + [
+        "//heron/statemgrs/src/java:null-statemgr-java",
+        "//heron/schedulers/src/java:null-scheduler-java",
+        "//heron/packing/src/java:roundrobin-packing",
+    ],
 )
 
 java_tests(
-  test_classes = [
-    "com.twitter.heron.scheduler.LaunchRunnerTest",
-    "com.twitter.heron.scheduler.RuntimeManagerRunnerTest",
-    "com.twitter.heron.scheduler.RuntimeManagerMainTest",
-    "com.twitter.heron.scheduler.SubmitterMainTest",
-    "com.twitter.heron.scheduler.SchedulerMainTest",
-    "com.twitter.heron.scheduler.UpdateTopologyManagerTest",
-    "com.twitter.heron.scheduler.client.LibrarySchedulerClientTest",
-    "com.twitter.heron.scheduler.client.HttpServiceSchedulerClientTest",
-    "com.twitter.heron.scheduler.client.SchedulerClientFactoryTest",
-    "com.twitter.heron.scheduler.dryrun.SubmitDryRunRenderTest",
-    "com.twitter.heron.scheduler.dryrun.UpdateDryRunRenderTest",
-    "com.twitter.heron.scheduler.server.SchedulerServerTest",
-    "com.twitter.heron.scheduler.utils.LauncherUtilsTest",
-    "com.twitter.heron.scheduler.utils.SchedulerUtilsTest",
-  ],
-  runtime_deps = [ ":scheduler-core-tests" ],
-  resources = ["//heron/scheduler-core/tests/resources:scheduler-resources"],
-  size = "small",
+    size = "small",
+    resources = ["//heron/scheduler-core/tests/resources:scheduler-resources"],
+    test_classes = [
+        "com.twitter.heron.scheduler.LaunchRunnerTest",
+        "com.twitter.heron.scheduler.RuntimeManagerRunnerTest",
+        "com.twitter.heron.scheduler.RuntimeManagerMainTest",
+        "com.twitter.heron.scheduler.SubmitterMainTest",
+        "com.twitter.heron.scheduler.SchedulerMainTest",
+        "com.twitter.heron.scheduler.UpdateTopologyManagerTest",
+        "com.twitter.heron.scheduler.client.LibrarySchedulerClientTest",
+        "com.twitter.heron.scheduler.client.HttpServiceSchedulerClientTest",
+        "com.twitter.heron.scheduler.client.SchedulerClientFactoryTest",
+        "com.twitter.heron.scheduler.dryrun.SubmitDryRunRenderTest",
+        "com.twitter.heron.scheduler.dryrun.UpdateDryRunRenderTest",
+        "com.twitter.heron.scheduler.dryrun.JsonFormatterUtilsTest",
+        "com.twitter.heron.scheduler.server.SchedulerServerTest",
+        "com.twitter.heron.scheduler.utils.LauncherUtilsTest",
+        "com.twitter.heron.scheduler.utils.SchedulerUtilsTest",
+    ],
+    runtime_deps = [":scheduler-core-tests"],
 )
diff --git a/heron/scheduler-core/tests/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtilsTest.java b/heron/scheduler-core/tests/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtilsTest.java
new file mode 100644
index 0000000000..492efbba62
--- /dev/null
+++ b/heron/scheduler-core/tests/java/com/twitter/heron/scheduler/dryrun/JsonFormatterUtilsTest.java
@@ -0,0 +1,70 @@
+//  Copyright 2018 Twitter. All rights reserved.
+//
+//  Licensed 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.
+package com.twitter.heron.scheduler.dryrun;
+
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.io.IOUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.twitter.heron.common.basics.Pair;
+import com.twitter.heron.spi.packing.PackingPlan;
+import com.twitter.heron.spi.utils.PackingTestUtils;
+
+import static com.twitter.heron.spi.packing.PackingPlan.ContainerPlan;
+import static junit.framework.TestCase.assertEquals;
+
+public class JsonFormatterUtilsTest {
+
+  private PackingPlan plan;
+
+  @Before
+  public void setUp() throws Exception {
+    final String COMPONENT_A = "exclaim1";
+    final String COMPONENT_B = "word";
+    ContainerPlan containerPlanA = PackingTestUtils.testContainerPlan(
+        1, new Pair<>(COMPONENT_A, 1),
+        new Pair<>(COMPONENT_A, 3),
+        new Pair<>(COMPONENT_B, 5));
+    ContainerPlan containerPlanB = PackingTestUtils.testContainerPlan(
+        2, new Pair<>(COMPONENT_A, 2),
+        new Pair<>(COMPONENT_A, 4),
+        new Pair<>(COMPONENT_B, 6));
+    Set<ContainerPlan> containerPlans = new HashSet<>();
+    containerPlans.add(containerPlanA);
+    containerPlans.add(containerPlanB);
+    plan = new PackingPlan("A", containerPlans);
+  }
+
+  @Test
+  public void testRenderPackingPlan() throws Exception {
+    JsonFormatterUtils utils = new JsonFormatterUtils();
+    String packingPlanJson = utils.renderPackingPlan("test-topology",
+        "com.twitter.heron.packing.roundrobin.RoundRobingPacking", plan);
+
+    String filename =  "/heron/scheduler-core/tests/resources/JsonFormatterUtilsExpectedJson.txt";
+    InputStream stream = JsonFormatterUtils.class.getResourceAsStream(filename);
+    if (stream == null) {
+      throw new RuntimeException("Expected json file cannot be found or opened.");
+    }
+    // Input might contain UTF-8 character, so we read stream with UTF-8 decoding
+    String expectedJson = IOUtils.toString(stream, StandardCharsets.UTF_8);
+
+    assertEquals(expectedJson, packingPlanJson);
+  }
+}
diff --git a/heron/scheduler-core/tests/resources/JsonFormatterUtilsExpectedJson.txt b/heron/scheduler-core/tests/resources/JsonFormatterUtilsExpectedJson.txt
new file mode 100644
index 0000000000..db9b92c0b2
--- /dev/null
+++ b/heron/scheduler-core/tests/resources/JsonFormatterUtilsExpectedJson.txt
@@ -0,0 +1 @@
+{"packingClass":"com.twitter.heron.packing.roundrobin.RoundRobingPacking","topology":"test-topology","containers":[{"id":1,"resources":{"cpu":6.0,"ram":7516192768,"disk":7516192768},"components":[{"component":"word","id":5,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}},{"component":"exclaim1","id":3,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}},{"component":"exclaim1","id":1,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}}]},{"id":2,"resources":{"cpu":6.0,"ram":7516192768,"disk":7516192768},"components":[{"component":"word","id":6,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}},{"component":"exclaim1","id":2,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}},{"component":"exclaim1","id":4,"index":1,"resources":{"cpu":1.5,"ram":2147483648,"disk":3}}]}]}
\ No newline at end of file
diff --git a/heron/tools/cli/src/python/args.py b/heron/tools/cli/src/python/args.py
index d70622e018..b17035703d 100644
--- a/heron/tools/cli/src/python/args.py
+++ b/heron/tools/cli/src/python/args.py
@@ -167,7 +167,7 @@ def add_dry_run(parser):
   :return:
   '''
   default_format = 'table'
-  resp_formats = ['raw', 'table', 'colored_table']
+  resp_formats = ['raw', 'table', 'colored_table', 'json']
   available_options = ', '.join(['%s' % opt for opt in resp_formats])
 
   def dry_run_resp_format(value):


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services