You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by th...@apache.org on 2016/10/28 14:47:54 UTC

[20/50] incubator-beam git commit: Add utility to handle JSON option manipulation

Add utility to handle JSON option manipulation


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/35244a68
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/35244a68
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/35244a68

Branch: refs/heads/apex-runner
Commit: 35244a680d53ddca4733dbef5b2570e6f78140d6
Parents: 7091a01
Author: sammcveety <sa...@gmail.com>
Authored: Fri Oct 14 13:06:08 2016 -0400
Committer: Dan Halperin <dh...@google.com>
Committed: Wed Oct 26 10:20:21 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/options/ValueProviderUtils.java    | 60 +++++++++++++++
 .../beam/sdk/options/ValueProviderTest.java     | 17 ++---
 .../sdk/options/ValueProviderUtilsTest.java     | 78 ++++++++++++++++++++
 3 files changed, 143 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
new file mode 100644
index 0000000..14a5f23
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.options;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Utilities for working with the {@link ValueProvider} interface.
+ */
+public class ValueProviderUtils {
+  private ValueProviderUtils() {}
+
+  /**
+   * Given {@code serializedOptions} as a JSON-serialized {@link PipelineOptions}, updates
+   * the values according to the provided values in {@code runtimeValues}.
+   */
+  public static String updateSerializedOptions(
+      String serializedOptions, Map<String, String> runtimeValues) {
+    ObjectMapper mapper = new ObjectMapper();
+    ObjectNode root, options;
+    try {
+      root = mapper.readValue(serializedOptions, ObjectNode.class);
+      options = (ObjectNode) root.get("options");
+      checkNotNull(options, "Unable to locate 'options' in %s", serializedOptions);
+    } catch (IOException e) {
+      throw new RuntimeException(
+        String.format("Unable to parse %s", serializedOptions), e);
+    }
+
+    for (Map.Entry<String, String> entry : runtimeValues.entrySet()) {
+      options.put(entry.getKey(), entry.getValue());
+    }
+    try {
+      return mapper.writeValueAsString(root);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse re-serialize options", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
index c9eb479..ed7a37a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java
@@ -17,14 +17,13 @@
  */
 package org.apache.beam.sdk.options;
 
-import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import java.util.List;
 import org.apache.beam.sdk.options.ValueProvider.RuntimeValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
@@ -182,12 +181,8 @@ public class ValueProviderTest {
     ObjectMapper mapper = new ObjectMapper();
     String serializedOptions = mapper.writeValueAsString(submitOptions);
 
-    // This is the expected behavior of the runner: deserialize and set the
-    // the runtime options.
-    String anchor = "\"appName\":\"ValueProviderTest\"";
-    assertThat(serializedOptions, containsString("\"foo\":null"));
-    String runnerString = serializedOptions.replaceAll(
-      "\"foo\":null", "\"foo\":\"quux\"");
+    String runnerString = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("foo", "quux"));
     TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class)
       .as(TestOptions.class);
 
@@ -206,10 +201,8 @@ public class ValueProviderTest {
     ObjectMapper mapper = new ObjectMapper();
     String serializedOptions = mapper.writeValueAsString(submitOptions);
 
-    // This is the expected behavior of the runner: deserialize and set the
-    // the runtime options.
-    assertThat(serializedOptions, containsString("baz"));
-    String runnerString = serializedOptions.replaceAll("baz", "quux");
+    String runnerString = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("foo", "quux"));
     TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class)
       .as(TestOptions.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/35244a68/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
new file mode 100644
index 0000000..0f8ed8e
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.options;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link ValueProviderUtils}. */
+@RunWith(JUnit4.class)
+public class ValueProviderUtilsTest {
+  /** A test interface. */
+  public interface TestOptions extends PipelineOptions {
+    String getString();
+    void setString(String value);
+
+    String getOtherString();
+    void setOtherString(String value);
+  }
+
+  @Test
+  public void testUpdateSerialize() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("string", "bar"));
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertEquals("bar", runtime.getString());
+  }
+
+  @Test
+  public void testUpdateSerializeExistingValue() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.fromArgs(
+      new String[]{"--string=baz", "--otherString=quux"}).as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.of("string", "bar"));
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertEquals("bar", runtime.getString());
+    assertEquals("quux", runtime.getOtherString());
+  }
+
+  @Test
+  public void testUpdateSerializeEmptyUpdate() throws Exception {
+    TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class);
+    ObjectMapper mapper = new ObjectMapper();
+    String serializedOptions = mapper.writeValueAsString(submitOptions);
+    String updatedOptions = ValueProviderUtils.updateSerializedOptions(
+      serializedOptions, ImmutableMap.<String, String>of());
+    TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class)
+      .as(TestOptions.class);
+    assertNull(runtime.getString());
+  }
+}