You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2016/12/07 00:02:51 UTC

[1/2] incubator-beam git commit: [BEAM-551] Add DisplayData handling of ValueProvider

Repository: incubator-beam
Updated Branches:
  refs/heads/master 5274dce5b -> b73311342


[BEAM-551] Add DisplayData handling of ValueProvider<String>


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

Branch: refs/heads/master
Commit: 2c8311f0eaabc130f09cb8566717ce8ea885ba97
Parents: 5274dce
Author: Sam McVeety <sg...@google.com>
Authored: Sat Dec 3 13:20:24 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Dec 6 15:14:14 2016 -0800

----------------------------------------------------------------------
 .../sdk/transforms/display/DisplayData.java     | 16 ++++
 .../sdk/transforms/display/DisplayDataTest.java | 86 ++++++++++++++++++++
 2 files changed, 102 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2c8311f0/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
index 1a2de97..f0040f7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
@@ -37,6 +37,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -863,6 +864,21 @@ public class DisplayData implements Serializable {
   }
 
   /**
+   * Create a display item for the specified key and {@link ValueProvider}.
+   */
+  public static ItemSpec<?> item(String key, ValueProvider<?> value) {
+    if (value.isAccessible()) {
+      Object got = value.get();
+      Type type = inferType(got);
+      if (type == null) {
+        throw new RuntimeException(String.format("Unknown value type: %s", got));
+      }
+      return item(key, type, got);
+    }
+    return item(key, Type.STRING, value.toString());
+  }
+
+  /**
    * Create a display item for the specified key and integer value.
    */
   public static ItemSpec<Integer> item(String key, @Nullable Integer value) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2c8311f0/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
index 770b836..fccd031 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
@@ -53,6 +53,8 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Map;
 import java.util.regex.Pattern;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData.Builder;
 import org.apache.beam.sdk.transforms.display.DisplayData.Item;
@@ -165,6 +167,90 @@ public class DisplayDataTest implements Serializable {
   }
 
   @Test
+  public void testStaticValueProviderDate() {
+    final Instant value = Instant.now();
+    DisplayData data =
+        DisplayData.from(new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add(DisplayData.item(
+                    "foo", StaticValueProvider.of(value)));
+              }
+            });
+
+    @SuppressWarnings("unchecked")
+    DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0];
+
+    @SuppressWarnings("unchecked")
+    Matcher<Item> matchesAllOf = Matchers.allOf(
+        hasKey("foo"),
+        hasType(DisplayData.Type.TIMESTAMP),
+        hasValue(ISO_FORMATTER.print(value)));
+
+    assertThat(item, matchesAllOf);
+  }
+
+  @Test
+  public void testStaticValueProviderString() {
+    DisplayData data =
+        DisplayData.from(new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add(DisplayData.item(
+                    "foo", StaticValueProvider.of("bar")));
+              }
+            });
+
+    assertThat(data.items(), hasSize(1));
+    assertThat(data, hasDisplayItem("foo", "bar"));
+  }
+
+  @Test
+  public void testStaticValueProviderInt() {
+    DisplayData data =
+        DisplayData.from(new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add(DisplayData.item(
+                    "foo", StaticValueProvider.of(1)));
+              }
+            });
+
+    assertThat(data.items(), hasSize(1));
+    assertThat(data, hasDisplayItem("foo", 1));
+  }
+
+  @Test
+  public void testInaccessibleValueProvider() {
+    DisplayData data =
+        DisplayData.from(new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add(DisplayData.item(
+                    "foo", new ValueProvider<String>() {
+                        @Override
+                        public boolean isAccessible() {
+                          return false;
+                        }
+
+                        @Override
+                        public String get() {
+                          return "bar";
+                        }
+
+                        @Override
+                        public String toString() {
+                          return "toString";
+                        }
+                      }));
+              }
+            });
+
+    assertThat(data.items(), hasSize(1));
+    assertThat(data, hasDisplayItem("foo", "toString"));
+  }
+
+  @Test
   public void testAsMap() {
     DisplayData data =
         DisplayData.from(


[2/2] incubator-beam git commit: This closes #1503

Posted by da...@apache.org.
This closes #1503


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

Branch: refs/heads/master
Commit: b733113426975acca7fb5c782ed92d1ecce9481e
Parents: 5274dce 2c8311f
Author: Davor Bonaci <da...@google.com>
Authored: Tue Dec 6 15:14:34 2016 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Dec 6 15:14:34 2016 -0800

----------------------------------------------------------------------
 .../sdk/transforms/display/DisplayData.java     | 16 ++++
 .../sdk/transforms/display/DisplayDataTest.java | 86 ++++++++++++++++++++
 2 files changed, 102 insertions(+)
----------------------------------------------------------------------