You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by bc...@apache.org on 2016/04/05 19:10:12 UTC

[1/2] incubator-beam git commit: DisplayData tweaks based on transform usage.

Repository: incubator-beam
Updated Branches:
  refs/heads/master 3e71ed481 -> 2f8ba65fa


DisplayData tweaks based on transform usage.

 * Add boolean-valued display data.
 * Implement equality for DislpayData.Item
 * Add ability to override namespace for included subcomponents.
 * Additional Matchers for testing display data
 * Update DisplayData inner class privacy


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

Branch: refs/heads/master
Commit: 014a9a5a2241b4f780efea30c14496961c55041d
Parents: 3e71ed4
Author: Scott Wegner <sw...@google.com>
Authored: Mon Mar 28 13:38:35 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Tue Apr 5 09:56:47 2016 -0700

----------------------------------------------------------------------
 .../sdk/transforms/display/DisplayData.java     |  85 +++++++--
 .../transforms/display/DisplayDataMatchers.java | 181 ++++++++++++++++++-
 .../display/DisplayDataMatchersTest.java        |  63 ++++++-
 .../sdk/transforms/display/DisplayDataTest.java | 128 ++++++++-----
 4 files changed, 391 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/014a9a5a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
index bfdf73e..63ec7fc 100644
--- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
@@ -88,7 +88,7 @@ public class DisplayData {
   public String toString() {
     StringBuilder builder = new StringBuilder();
     boolean isFirstLine = true;
-    for (Map.Entry<Identifier, Item> entry : entries.entrySet()) {
+    for (Item entry : entries.values()) {
       if (isFirstLine) {
         isFirstLine = false;
       } else {
@@ -107,14 +107,19 @@ public class DisplayData {
    */
   public interface Builder {
     /**
-     * Include display metadata from the specified subcomponent. For example, a {@link ParDo}
+     * Register display metadata from the specified subcomponent. For example, a {@link ParDo}
      * transform includes display metadata from the encapsulated {@link DoFn}.
-     *
-     * @return A builder instance to continue to build in a fluent-style.
      */
     Builder include(HasDisplayData subComponent);
 
     /**
+     * Register display metadata from the specified subcomponent, using the specified namespace.
+     * For example, a {@link ParDo} transform includes display metadata from the encapsulated
+     * {@link DoFn}.
+     */
+    Builder include(HasDisplayData subComponent, Class<?> namespace);
+
+    /**
      * Register the given string display metadata. The metadata item will be registered with type
      * {@link DisplayData.Type#STRING}, and is identified by the specified key and namespace from
      * the current transform or component.
@@ -136,6 +141,13 @@ public class DisplayData {
     ItemBuilder add(String key, double value);
 
     /**
+     * Register the given floating point display metadata. The metadata item will be registered with
+     * type {@link DisplayData.Type#BOOLEAN}, and is identified by the specified key and namespace
+     * from the current transform or component.
+     */
+    ItemBuilder add(String key, boolean value);
+
+    /**
      * Register the given timestamp display metadata. The metadata item will be registered with type
      * {@link DisplayData.Type#TIMESTAMP}, and is identified by the specified key and namespace from
      * the current transform or component.
@@ -287,7 +299,35 @@ public class DisplayData {
 
     @Override
     public String toString() {
-      return getValue();
+      return String.format("%s:%s=%s", ns, key, value);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof Item) {
+        Item that = (Item) obj;
+        return Objects.equals(this.ns, that.ns)
+            && Objects.equals(this.key, that.key)
+            && Objects.equals(this.type, that.type)
+            && Objects.equals(this.value, that.value)
+            && Objects.equals(this.shortValue, that.shortValue)
+            && Objects.equals(this.label, that.label)
+            && Objects.equals(this.url, that.url);
+      }
+
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(
+          this.ns,
+          this.key,
+          this.type,
+          this.value,
+          this.shortValue,
+          this.label,
+          this.url);
     }
 
     private Item withLabel(String label) {
@@ -313,8 +353,12 @@ public class DisplayData {
     private final String ns;
     private final String key;
 
-    static Identifier of(Class<?> namespace, String key) {
-      return new Identifier(namespace.getName(), key);
+    public static Identifier of(Class<?> namespace, String key) {
+      return of(namespace.getName(), key);
+    }
+
+    public static Identifier of(String namespace, String key) {
+      return new Identifier(namespace, key);
     }
 
     private Identifier(String ns, String key) {
@@ -355,7 +399,7 @@ public class DisplayData {
   /**
    * Display metadata type.
    */
-  enum Type {
+  public enum Type {
     STRING {
       @Override
       FormattedItemValue format(Object value) {
@@ -374,6 +418,12 @@ public class DisplayData {
         return new FormattedItemValue(Double.toString((Double) value));
       }
     },
+    BOOLEAN() {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue(Boolean.toString((boolean) value));
+      }
+    },
     TIMESTAMP() {
       @Override
       FormattedItemValue format(Object value) {
@@ -403,7 +453,7 @@ public class DisplayData {
     abstract FormattedItemValue format(Object value);
   }
 
-  private static class FormattedItemValue {
+  static class FormattedItemValue {
     private final String shortValue;
     private final String longValue;
 
@@ -416,11 +466,11 @@ public class DisplayData {
       this.shortValue = shortValue;
     }
 
-    private String getLongValue () {
+    String getLongValue() {
       return this.longValue;
     }
 
-    private String getShortValue() {
+    String getShortValue() {
       return this.shortValue;
     }
   }
@@ -447,10 +497,16 @@ public class DisplayData {
     @Override
     public Builder include(HasDisplayData subComponent) {
       checkNotNull(subComponent);
+      return include(subComponent, subComponent.getClass());
+    }
+
+    @Override
+    public Builder include(HasDisplayData subComponent, Class<?> namespace) {
+      checkNotNull(subComponent);
       boolean newComponent = visited.add(subComponent);
       if (newComponent) {
         Class prevNs = this.latestNs;
-        this.latestNs = subComponent.getClass();
+        this.latestNs = namespace;
         subComponent.populateDisplayData(this);
         this.latestNs = prevNs;
       }
@@ -475,6 +531,11 @@ public class DisplayData {
     }
 
     @Override
+    public ItemBuilder add(String key, boolean value) {
+      return addItem(key, Type.BOOLEAN, value);
+    }
+
+    @Override
     public ItemBuilder add(String key, Instant value) {
       checkNotNull(value);
       return addItem(key, Type.TIMESTAMP, value);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/014a9a5a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
index 385bc42..d540b4b 100644
--- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
+++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
@@ -17,13 +17,19 @@
  */
 package com.google.cloud.dataflow.sdk.transforms.display;
 
+import static org.hamcrest.Matchers.allOf;
+
 import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
 
+import com.google.common.collect.Sets;
+import org.hamcrest.CustomTypeSafeMatcher;
 import org.hamcrest.Description;
 import org.hamcrest.FeatureMatcher;
 import org.hamcrest.Matcher;
 import org.hamcrest.Matchers;
 import org.hamcrest.TypeSafeDiagnosingMatcher;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
 
 import java.util.Collection;
 
@@ -44,6 +50,71 @@ public class DisplayDataMatchers {
   }
 
   /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and String value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, String value) {
+    return hasDisplayItem(key, DisplayData.Type.STRING, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Boolean value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, Boolean value) {
+    return hasDisplayItem(key, DisplayData.Type.BOOLEAN, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Duration value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, Duration value) {
+    return hasDisplayItem(key, DisplayData.Type.DURATION, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Float value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, double value) {
+    return hasDisplayItem(key, DisplayData.Type.FLOAT, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Integer value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, long value) {
+    return hasDisplayItem(key, DisplayData.Type.INTEGER, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Class value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, Class<?> value) {
+    return hasDisplayItem(key, DisplayData.Type.JAVA_CLASS, value);
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains an item with the
+   * specified key and Timestamp value.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(String key, Instant value) {
+    return hasDisplayItem(key, DisplayData.Type.TIMESTAMP, value);
+  }
+
+  private static Matcher<DisplayData> hasDisplayItem(
+      String key, DisplayData.Type type, Object value) {
+    DisplayData.FormattedItemValue formattedValue = type.format(value);
+    return hasDisplayItem(allOf(
+        hasKey(key),
+        hasType(type),
+        hasValue(formattedValue.getLongValue())));
+  }
+
+  /**
    * Creates a matcher that matches if the examined {@link DisplayData} contains any item
    * matching the specified {@code itemMatcher}.
    */
@@ -69,7 +140,8 @@ public class DisplayDataMatchers {
       Collection<Item> items = data.items();
       boolean isMatch = Matchers.hasItem(itemMatcher).matches(items);
       if (!isMatch) {
-        mismatchDescription.appendText("found " + items.size() + " non-matching items");
+        mismatchDescription.appendText("found " + items.size() + " non-matching item(s):\n");
+        mismatchDescription.appendValue(data);
       }
 
       return isMatch;
@@ -77,6 +149,85 @@ public class DisplayDataMatchers {
   }
 
   /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains all display data
+   * registered from the specified subcomponent.
+   */
+  public static Matcher<DisplayData> includes(final HasDisplayData subComponent) {
+    return includes(subComponent, subComponent.getClass());
+  }
+
+  /**
+   * Create a matcher that matches if the examined {@link DisplayData} contains all display data
+   * registered from the specified subcomponent and namespace.
+   */
+  public static Matcher<DisplayData> includes(
+      final HasDisplayData subComponent, final Class<? extends HasDisplayData> namespace) {
+    return new CustomTypeSafeMatcher<DisplayData>("includes subcomponent") {
+      @Override
+      protected boolean matchesSafely(DisplayData displayData) {
+        DisplayData subComponentData = DisplayData.from(subComponent);
+        if (subComponentData.items().size() == 0) {
+          throw new UnsupportedOperationException("subComponent contains no display data; " +
+              "cannot verify whether it is included");
+        }
+
+        DisplayDataComparision comparison = checkSubset(displayData, subComponentData, namespace);
+        return comparison.missingItems.isEmpty();
+      }
+
+
+      @Override
+      protected void describeMismatchSafely(
+          DisplayData displayData, Description mismatchDescription) {
+        DisplayData subComponentDisplayData = DisplayData.from(subComponent);
+        DisplayDataComparision comparison = checkSubset(
+            displayData, subComponentDisplayData, subComponent.getClass());
+
+        mismatchDescription
+            .appendText("did not include:\n")
+            .appendValue(comparison.missingItems)
+            .appendText("\nNon-matching items:\n")
+            .appendValue(comparison.unmatchedItems);
+      }
+
+      private DisplayDataComparision checkSubset(
+          DisplayData displayData, DisplayData included, Class<?> namespace) {
+        DisplayDataComparision comparison = new DisplayDataComparision(displayData.items());
+        for (Item item : included.items()) {
+          Item matchedItem = displayData.asMap().get(
+              DisplayData.Identifier.of(namespace, item.getKey()));
+
+          if (matchedItem != null) {
+            comparison.matched(matchedItem);
+          } else {
+            comparison.missing(item);
+          }
+        }
+
+        return comparison;
+      }
+
+      class DisplayDataComparision {
+        Collection<DisplayData.Item> missingItems;
+        Collection<DisplayData.Item> unmatchedItems;
+
+        DisplayDataComparision(Collection<Item> superset) {
+          missingItems = Sets.newHashSet();
+          unmatchedItems = Sets.newHashSet(superset);
+        }
+
+        void matched(Item supersetItem) {
+          unmatchedItems.remove(supersetItem);
+        }
+
+        void missing(Item subsetItem) {
+          missingItems.add(subsetItem);
+        }
+      }
+    };
+  }
+
+  /**
    * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key
    * with the specified value.
    */
@@ -96,4 +247,32 @@ public class DisplayDataMatchers {
       }
     };
   }
+
+  public static Matcher<DisplayData.Item> hasType(DisplayData.Type type) {
+    return hasType(Matchers.is(type));
+  }
+
+  public static Matcher<DisplayData.Item> hasType(Matcher<DisplayData.Type> typeMatcher) {
+    return new FeatureMatcher<DisplayData.Item, DisplayData.Type>(
+            typeMatcher, "with type", "type") {
+      @Override
+      protected DisplayData.Type featureValueOf(DisplayData.Item actual) {
+        return actual.getType();
+      }
+    };
+  }
+
+  public static Matcher<DisplayData.Item> hasValue(String value) {
+    return hasValue(Matchers.is(value));
+  }
+
+  public static Matcher<DisplayData.Item> hasValue(Matcher<String> valueMatcher) {
+    return new FeatureMatcher<DisplayData.Item, String>(
+            valueMatcher, "with value", "value") {
+      @Override
+      protected String featureValueOf(DisplayData.Item actual) {
+        return actual.getValue();
+      }
+    };
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/014a9a5a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
index 3ade923..1b43ff7 100644
--- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
+++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
@@ -19,11 +19,13 @@ package com.google.cloud.dataflow.sdk.transforms.display;
 
 import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.core.StringStartsWith.startsWith;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
 
 import com.google.cloud.dataflow.sdk.transforms.PTransform;
 import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
@@ -46,7 +48,7 @@ public class DisplayDataMatchersTest {
     Matcher<DisplayData> matcher = hasDisplayItem();
 
     assertFalse(matcher.matches(DisplayData.none()));
-    assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar")));
+    assertThat(createDisplayDataWithItem("foo", "bar"), matcher);
   }
 
   @Test
@@ -59,15 +61,68 @@ public class DisplayDataMatchersTest {
     matcher.describeMismatch(DisplayData.none(), mismatchDesc);
 
     assertThat(desc.toString(), startsWith("display data with item: "));
-    assertThat(mismatchDesc.toString(), containsString("found 0 non-matching items"));
+    assertThat(mismatchDesc.toString(), containsString("found 0 non-matching item(s)"));
   }
 
   @Test
   public void testHasKey() {
     Matcher<DisplayData> matcher = hasDisplayItem(hasKey("foo"));
 
-    assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar")));
     assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar")));
+
+    assertThat(createDisplayDataWithItem("foo", "bar"), matcher);
+  }
+
+  @Test
+  public void testHasType() {
+    Matcher<DisplayData> matcher = hasDisplayItem(hasType(DisplayData.Type.JAVA_CLASS));
+
+    DisplayData data = DisplayData.from(new PTransform<PCollection<String>, PCollection<String>>() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder.add("foo", DisplayDataMatchersTest.class);
+      }
+    });
+
+    assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar")));
+    assertThat(data, matcher);
+  }
+
+  @Test
+  public void testHasValue() {
+    Matcher<DisplayData> matcher = hasDisplayItem(hasValue("bar"));
+
+    assertFalse(matcher.matches(createDisplayDataWithItem("foo", "baz")));
+    assertThat(createDisplayDataWithItem("foo", "bar"), matcher);
+  }
+
+  @Test
+  public void testIncludes() {
+    final HasDisplayData subComponent = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder.add("foo", "bar");
+      }
+    };
+    HasDisplayData hasSubcomponent = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder
+          .include(subComponent)
+          .add("foo2", "bar2");
+      }
+    };
+    HasDisplayData sameKeyDifferentNamespace = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder.add("foo", "bar");
+      }
+    };
+    Matcher<DisplayData> matcher = includes(subComponent);
+
+    assertFalse(matcher.matches(DisplayData.from(sameKeyDifferentNamespace)));
+    assertThat(DisplayData.from(hasSubcomponent), matcher);
+    assertThat(DisplayData.from(subComponent), matcher);
   }
 
   private DisplayData createDisplayDataWithItem(final String key, final String value) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/014a9a5a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
index 397e102..4d75e26 100644
--- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
+++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
@@ -19,6 +19,9 @@ package com.google.cloud.dataflow.sdk.transforms.display;
 
 import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue;
+import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.everyItem;
@@ -92,7 +95,7 @@ public class DisplayDataTest {
                 .include(subComponent2)
                 .add("MinSproggles", 200)
                 .withLabel("Mimimum Required Sproggles")
-                .add("LazerOrientation", "NORTH")
+                .add("FireLazers", true)
                 .add("TimeBomb", Instant.now().plus(Duration.standardDays(1)))
                 .add("FilterLogic", subComponent1.getClass())
                 .add("ServiceUrl", "google.com/fizzbang")
@@ -127,12 +130,12 @@ public class DisplayDataTest {
         DisplayData.from(new HasDisplayData() {
               @Override
               public void populateDisplayData(DisplayData.Builder builder) {
-                builder.add("Foo", "bar");
+                builder.add("foo", "bar");
               }
             });
 
     assertThat(data.items(), hasSize(1));
-    assertThat(data, hasDisplayItem(hasKey("Foo")));
+    assertThat(data, hasDisplayItem("foo", "bar"));
   }
 
   @Test
@@ -148,7 +151,7 @@ public class DisplayDataTest {
 
     Map<DisplayData.Identifier, DisplayData.Item> map = data.asMap();
     assertEquals(map.size(), 1);
-    assertThat(data, hasDisplayItem(hasKey("foo")));
+    assertThat(data, hasDisplayItem("foo", "bar"));
     assertEquals(map.values(), data.items());
   }
 
@@ -164,8 +167,8 @@ public class DisplayDataTest {
         allOf(
             hasNamespace(Matchers.<Class<?>>is(ConcreteComponent.class)),
             hasKey("now"),
-            hasType(is(DisplayData.Type.TIMESTAMP)),
-            hasValue(is(ISO_FORMATTER.print(value))),
+            hasType(DisplayData.Type.TIMESTAMP),
+            hasValue(ISO_FORMATTER.print(value)),
             hasShortValue(nullValue(String.class)),
             hasLabel(is("the current instant")),
             hasUrl(is("http://time.gov"))));
@@ -219,14 +222,35 @@ public class DisplayDataTest {
               }
             });
 
-    assertThat(
-        data,
-        hasDisplayItem(
-            allOf(
-                hasKey("foo"),
-                hasNamespace(Matchers.<Class<?>>is(subComponent.getClass())))));
+    assertThat(data, includes(subComponent));
+  }
+
+  @Test
+  public void testIncludesNamespaceOverride() {
+    final HasDisplayData subComponent = new HasDisplayData() {
+        @Override
+        public void populateDisplayData(DisplayData.Builder builder) {
+          builder.add("foo", "bar");
+        }
+    };
+
+    final HasDisplayData namespaceOverride = new HasDisplayData(){
+      @Override
+      public void populateDisplayData(Builder builder) {
+      }
+    };
+
+    DisplayData data = DisplayData.from(new HasDisplayData() {
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder.include(subComponent, namespaceOverride.getClass());
+      }
+    });
+
+    assertThat(data, includes(subComponent, namespaceOverride.getClass()));
   }
 
+
   @Test
   public void testIdentifierEquality() {
     new EqualsTester()
@@ -239,6 +263,33 @@ public class DisplayDataTest {
   }
 
   @Test
+  public void testItemEquality() {
+    HasDisplayData component1 = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder.add("foo", "bar");
+      }
+    };
+    HasDisplayData component2 = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(Builder builder) {
+        builder.add("foo", "bar");
+      }
+    };
+
+    DisplayData component1DisplayData1 = DisplayData.from(component1);
+    DisplayData component1DisplayData2 = DisplayData.from(component1);
+    DisplayData component2DisplayData = DisplayData.from(component2);
+
+    new EqualsTester()
+        .addEqualityGroup(
+            component1DisplayData1.items().toArray()[0],
+            component1DisplayData2.items().toArray()[0])
+        .addEqualityGroup(component2DisplayData.items().toArray()[0])
+        .testEquals();
+  }
+
+  @Test
   public void testAnonymousClassNamespace() {
     DisplayData data =
         DisplayData.from(
@@ -404,6 +455,7 @@ public class DisplayDataTest {
                     .add("string", "foobar")
                     .add("integer", 123)
                     .add("float", 3.14)
+                    .add("boolean", true)
                     .add("java_class", DisplayDataTest.class)
                     .add("timestamp", Instant.now())
                     .add("duration", Duration.standardHours(1));
@@ -412,18 +464,19 @@ public class DisplayDataTest {
 
     Collection<Item> items = data.items();
     assertThat(
-        items, hasItem(allOf(hasKey("string"), hasType(is(DisplayData.Type.STRING)))));
+        items, hasItem(allOf(hasKey("string"), hasType(DisplayData.Type.STRING))));
     assertThat(
-        items, hasItem(allOf(hasKey("integer"), hasType(is(DisplayData.Type.INTEGER)))));
-    assertThat(items, hasItem(allOf(hasKey("float"), hasType(is(DisplayData.Type.FLOAT)))));
+        items, hasItem(allOf(hasKey("integer"), hasType(DisplayData.Type.INTEGER))));
+    assertThat(items, hasItem(allOf(hasKey("float"), hasType(DisplayData.Type.FLOAT))));
+    assertThat(items, hasItem(allOf(hasKey("boolean"), hasType(DisplayData.Type.BOOLEAN))));
     assertThat(
         items,
-        hasItem(allOf(hasKey("java_class"), hasType(is(DisplayData.Type.JAVA_CLASS)))));
+        hasItem(allOf(hasKey("java_class"), hasType(DisplayData.Type.JAVA_CLASS))));
     assertThat(
         items,
-        hasItem(allOf(hasKey("timestamp"), hasType(is(DisplayData.Type.TIMESTAMP)))));
+        hasItem(allOf(hasKey("timestamp"), hasType(DisplayData.Type.TIMESTAMP))));
     assertThat(
-        items, hasItem(allOf(hasKey("duration"), hasType(is(DisplayData.Type.DURATION)))));
+        items, hasItem(allOf(hasKey("duration"), hasType(DisplayData.Type.DURATION))));
   }
 
   @Test
@@ -438,6 +491,7 @@ public class DisplayDataTest {
           .add("string", "foobar")
           .add("integer", 123)
           .add("float", 3.14)
+          .add("boolean", true)
           .add("java_class", DisplayDataTest.class)
           .add("timestamp", now)
           .add("duration", oneHour);
@@ -445,17 +499,13 @@ public class DisplayDataTest {
     };
     DisplayData data = DisplayData.from(component);
 
-    Collection<Item> items = data.items();
-    assertThat(items, hasItem(allOf(hasKey("string"), hasValue(is("foobar")))));
-    assertThat(items, hasItem(allOf(hasKey("integer"), hasValue(is("123")))));
-    assertThat(items, hasItem(allOf(hasKey("float"), hasValue(is("3.14")))));
-    assertThat(items, hasItem(allOf(hasKey("java_class"),
-            hasValue(is(DisplayDataTest.class.getName())),
-            hasShortValue(is(DisplayDataTest.class.getSimpleName())))));
-    assertThat(items, hasItem(allOf(hasKey("timestamp"),
-            hasValue(is(ISO_FORMATTER.print(now))))));
-    assertThat(items, hasItem(allOf(hasKey("duration"),
-            hasValue(is(Long.toString(oneHour.getMillis()))))));
+    assertThat(data, hasDisplayItem("string", "foobar"));
+    assertThat(data, hasDisplayItem("integer", 123));
+    assertThat(data, hasDisplayItem("float", 3.14));
+    assertThat(data, hasDisplayItem("boolean", true));
+    assertThat(data, hasDisplayItem("java_class", DisplayDataTest.class));
+    assertThat(data, hasDisplayItem("timestamp", now));
+    assertThat(data, hasDisplayItem("duration", oneHour));
   }
 
   @Test
@@ -582,16 +632,6 @@ public class DisplayDataTest {
     };
   }
 
-  private static Matcher<DisplayData.Item> hasType(Matcher<DisplayData.Type> typeMatcher) {
-    return new FeatureMatcher<DisplayData.Item, DisplayData.Type>(
-        typeMatcher, "display item with type", "type") {
-      @Override
-      protected DisplayData.Type featureValueOf(DisplayData.Item actual) {
-        return actual.getType();
-      }
-    };
-  }
-
   private static Matcher<DisplayData.Item> hasLabel(Matcher<String> labelMatcher) {
     return new FeatureMatcher<DisplayData.Item, String>(
         labelMatcher, "display item with label", "label") {
@@ -612,16 +652,6 @@ public class DisplayDataTest {
     };
   }
 
-  private static Matcher<DisplayData.Item> hasValue(Matcher<String> valueMatcher) {
-    return new FeatureMatcher<DisplayData.Item, String>(
-        valueMatcher, "display item with value", "value") {
-      @Override
-      protected String featureValueOf(DisplayData.Item actual) {
-        return actual.getValue();
-      }
-    };
-  }
-
   private static Matcher<DisplayData.Item> hasShortValue(Matcher<String> valueStringMatcher) {
     return new FeatureMatcher<DisplayData.Item, String>(
         valueStringMatcher, "display item with short value", "short value") {


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

Posted by bc...@apache.org.
This closes #83


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

Branch: refs/heads/master
Commit: 2f8ba65fa84fa3aee9cc235d94c2759fdda99795
Parents: 3e71ed4 014a9a5
Author: bchambers <bc...@google.com>
Authored: Tue Apr 5 09:58:34 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Tue Apr 5 09:58:34 2016 -0700

----------------------------------------------------------------------
 .../sdk/transforms/display/DisplayData.java     |  85 +++++++--
 .../transforms/display/DisplayDataMatchers.java | 181 ++++++++++++++++++-
 .../display/DisplayDataMatchersTest.java        |  63 ++++++-
 .../sdk/transforms/display/DisplayDataTest.java | 128 ++++++++-----
 4 files changed, 391 insertions(+), 66 deletions(-)
----------------------------------------------------------------------