You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by mx...@apache.org on 2016/03/23 19:35:08 UTC

[03/10] incubator-beam git commit: Add DisplayData builder API to SDK

Add DisplayData builder API to SDK

This allows generating the display data which will be attached to
PTransforms.


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

Branch: refs/heads/master
Commit: 5ecb7aa7a8ac107e2bdb8518da2bee714ceba122
Parents: cb5d6c2
Author: Scott Wegner <sw...@google.com>
Authored: Thu Mar 17 10:22:42 2016 -0700
Committer: Maximilian Michels <mx...@apache.org>
Committed: Wed Mar 23 19:27:51 2016 +0100

----------------------------------------------------------------------
 sdk/pom.xml                                     |   7 +
 .../cloud/dataflow/sdk/transforms/DoFn.java     |  13 +-
 .../dataflow/sdk/transforms/PTransform.java     |  14 +-
 .../cloud/dataflow/sdk/transforms/ParDo.java    |  13 +
 .../sdk/transforms/display/DisplayData.java     | 517 +++++++++++++++
 .../sdk/transforms/display/HasDisplayData.java  |  53 ++
 .../cloud/dataflow/sdk/transforms/DoFnTest.java |  15 +
 .../dataflow/sdk/transforms/PTransformTest.java |  41 ++
 .../dataflow/sdk/transforms/ParDoTest.java      |  23 +
 .../transforms/display/DisplayDataMatchers.java |  98 +++
 .../display/DisplayDataMatchersTest.java        |  81 +++
 .../sdk/transforms/display/DisplayDataTest.java | 633 +++++++++++++++++++
 .../cloud/dataflow/sdk/util/ApiSurfaceTest.java |   3 +-
 13 files changed, 1508 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/sdk/pom.xml b/sdk/pom.xml
index 71f5097..185abc2 100644
--- a/sdk/pom.xml
+++ b/sdk/pom.xml
@@ -615,6 +615,13 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava-testlib</artifactId>
+      <version>${guava.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
       <version>${protobuf.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java
index af06cc8..5ba9992 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java
@@ -24,6 +24,8 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental;
 import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind;
 import com.google.cloud.dataflow.sdk.options.PipelineOptions;
 import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData;
 import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
 import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
 import com.google.cloud.dataflow.sdk.util.WindowingInternals;
@@ -69,7 +71,7 @@ import java.util.UUID;
  * @param <InputT> the type of the (main) input elements
  * @param <OutputT> the type of the (main) output elements
  */
-public abstract class DoFn<InputT, OutputT> implements Serializable {
+public abstract class DoFn<InputT, OutputT> implements Serializable, HasDisplayData {
 
   /**
    * Information accessible to all methods in this {@code DoFn}.
@@ -366,6 +368,15 @@ public abstract class DoFn<InputT, OutputT> implements Serializable {
   public void finishBundle(Context c) throws Exception {
   }
 
+  /**
+   * {@inheritDoc}
+   *
+   * <p>By default, does not register any display data. Implementors may override this method
+   * to provide their own display metadata.
+   */
+  @Override
+  public void populateDisplayData(DisplayData.Builder builder) {
+  }
 
   /////////////////////////////////////////////////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java
index 8a74509..d4496b8 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java
@@ -19,6 +19,8 @@ package com.google.cloud.dataflow.sdk.transforms;
 import com.google.cloud.dataflow.sdk.Pipeline;
 import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
 import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
+import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData;
 import com.google.cloud.dataflow.sdk.util.StringUtils;
 import com.google.cloud.dataflow.sdk.values.PInput;
 import com.google.cloud.dataflow.sdk.values.POutput;
@@ -168,7 +170,7 @@ import java.io.Serializable;
  * @param <OutputT> the type of the output of this PTransform
  */
 public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
-    implements Serializable /* See the note above */ {
+    implements Serializable /* See the note above */, HasDisplayData {
   /**
    * Applies this {@code PTransform} on the given {@code InputT}, and returns its
    * {@code Output}.
@@ -309,4 +311,14 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
     Coder<T> defaultOutputCoder = (Coder<T>) getDefaultOutputCoder(input);
     return defaultOutputCoder;
   }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>By default, does not register any display data. Implementors may override this method
+   * to provide their own display metadata.
+   */
+  @Override
+  public void populateDisplayData(Builder builder) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java
index 0922767..c77ac44 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java
@@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
 import com.google.cloud.dataflow.sdk.coders.Coder;
 import com.google.cloud.dataflow.sdk.coders.CoderException;
 import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
 import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
 import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext;
 import com.google.cloud.dataflow.sdk.util.DirectSideInputReader;
@@ -787,6 +788,18 @@ public class ParDo {
       }
     }
 
+    /**
+     * {@inheritDoc}
+     *
+     * <p>{@link ParDo} registers its internal {@link DoFn} as a subcomponent for display metadata.
+     * {@link DoFn} implementations can register display data by overriding
+     * {@link DoFn#populateDisplayData}.
+     */
+    @Override
+    public void populateDisplayData(Builder builder) {
+      builder.include(fn);
+    }
+
     public DoFn<InputT, OutputT> getFn() {
       return fn;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
new file mode 100644
index 0000000..05fa7c7
--- /dev/null
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java
@@ -0,0 +1,517 @@
+/*
+ * Copyright (C) 2016 Google Inc.
+ *
+ * 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.google.cloud.dataflow.sdk.transforms.display;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.avro.reflect.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Static display metadata associated with a pipeline component. Display data is useful for
+ * pipeline runner UIs and diagnostic dashboards to display details about
+ * {@link PTransform PTransforms} that make up a pipeline.
+ *
+ * <p>Components specify their display data by implementing the {@link HasDisplayData}
+ * interface.
+ */
+public class DisplayData {
+  private static final DisplayData EMPTY = new DisplayData(Maps.<Identifier, Item>newHashMap());
+  private static final DateTimeFormatter TIMESTAMP_FORMATTER = ISODateTimeFormat.dateTime();
+
+  private final ImmutableMap<Identifier, Item> entries;
+
+  private DisplayData(Map<Identifier, Item> entries) {
+    this.entries = ImmutableMap.copyOf(entries);
+  }
+
+  /**
+   * Default empty {@link DisplayData} instance.
+   */
+  public static DisplayData none() {
+    return EMPTY;
+  }
+
+  /**
+   * Collect the {@link DisplayData} from a component. This will traverse all subcomponents
+   * specified via {@link Builder#include} in the given component. Data in this component will be in
+   * a namespace derived from the component.
+   */
+  public static DisplayData from(HasDisplayData component) {
+    checkNotNull(component);
+    return InternalBuilder.forRoot(component).build();
+  }
+
+  public Collection<Item> items() {
+    return entries.values();
+  }
+
+  public Map<Identifier, Item> asMap() {
+    return entries;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    boolean isFirstLine = true;
+    for (Map.Entry<Identifier, Item> entry : entries.entrySet()) {
+      if (isFirstLine) {
+        isFirstLine = false;
+      } else {
+        builder.append("\n");
+      }
+
+      builder.append(entry);
+    }
+
+    return builder.toString();
+  }
+
+  /**
+   * Utility to build up display metadata from a component and its included
+   * subcomponents.
+   */
+  public interface Builder {
+    /**
+     * Include 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 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.
+     */
+    ItemBuilder add(String key, String value);
+
+    /**
+     * Register the given numeric display metadata. The metadata item will be registered with type
+     * {@link DisplayData.Type#INTEGER}, and is identified by the specified key and namespace from
+     * the current transform or component.
+     */
+    ItemBuilder add(String key, long value);
+
+    /**
+     * Register the given floating point display metadata. The metadata item will be registered with
+     * type {@link DisplayData.Type#FLOAT}, and is identified by the specified key and namespace
+     * from the current transform or component.
+     */
+    ItemBuilder add(String key, double 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.
+     */
+    ItemBuilder add(String key, Instant value);
+
+    /**
+     * Register the given duration display metadata. The metadata item will be registered with type
+     * {@link DisplayData.Type#DURATION}, and is identified by the specified key and namespace from
+     * the current transform or component.
+     */
+    ItemBuilder add(String key, Duration value);
+
+    /**
+     * Register the given class display metadata. The metadata item will be registered with type
+     * {@link DisplayData.Type#JAVA_CLASS}, and is identified by the specified key and namespace
+     * from the current transform or component.
+     */
+    ItemBuilder add(String key, Class<?> value);
+  }
+
+  /**
+   * Utility to append optional fields to display metadata, or register additional display metadata
+   * items.
+   */
+  public interface ItemBuilder extends Builder {
+    /**
+     * Add a human-readable label to describe the most-recently added metadata field.
+     * A label is optional; if unspecified, UIs should display the metadata key to identify the
+     * display item.
+     *
+     * <p>Specifying a null value will clear the label if it was previously defined.
+     */
+    ItemBuilder withLabel(@Nullable String label);
+
+    /**
+     * Add a link URL to the most-recently added display metadata. A link URL is optional and
+     * can be provided to point the reader to additional details about the metadata.
+     *
+     * <p>Specifying a null value will clear the URL if it was previously defined.
+     */
+    ItemBuilder withLinkUrl(@Nullable String url);
+  }
+
+  /**
+   * A display metadata item. DisplayData items are registered via {@link Builder#add} within
+   * {@link HasDisplayData#populateDisplayData} implementations. Each metadata item is uniquely
+   * identified by the specified key and namespace generated from the registering component's
+   * class name.
+   */
+  public static class Item {
+    private final String key;
+    private final String ns;
+    private final Type type;
+    private final String value;
+    private final String shortValue;
+    private final String label;
+    private final String url;
+
+    private static <T> Item create(String namespace, String key, Type type, T value) {
+      FormattedItemValue formatted = type.format(value);
+      return new Item(
+        namespace, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null);
+    }
+
+    private Item(
+        String namespace,
+        String key,
+        Type type,
+        String value,
+        String shortValue,
+        String url,
+        String label) {
+      this.ns = namespace;
+      this.key = key;
+      this.type = type;
+      this.value = value;
+      this.shortValue = shortValue;
+      this.url = url;
+      this.label = label;
+    }
+
+    public String getNamespace() {
+      return ns;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    /**
+     * Retrieve the {@link DisplayData.Type} of display metadata. All metadata conforms to a
+     * predefined set of allowed types.
+     */
+    public Type getType() {
+      return type;
+    }
+
+    /**
+     * Retrieve the value of the metadata item.
+     */
+    public String getValue() {
+      return value;
+    }
+
+    /**
+     * Return the optional short value for an item. Types may provide a short-value to displayed
+     * instead of or in addition to the full {@link Item#value}.
+     *
+     * <p>Some display data types will not provide a short value, in which case the return value
+     * will be null.
+     */
+    @Nullable
+    public String getShortValue() {
+      return shortValue;
+    }
+
+    /**
+     * Retrieve the optional label for an item. The label is a human-readable description of what
+     * the metadata represents. UIs may choose to display the label instead of the item key.
+     *
+     * <p>If no label was specified, this will return {@code null}.
+     */
+    @Nullable
+    public String getLabel() {
+      return label;
+    }
+
+    /**
+     * Retrieve the optional link URL for an item. The URL points to an address where the reader
+     * can find additional context for the display metadata.
+     *
+     * <p>If no URL was specified, this will return {@code null}.
+     */
+    @Nullable
+    public String getUrl() {
+      return url;
+    }
+
+    @Override
+    public String toString() {
+      return getValue();
+    }
+
+    private Item withLabel(String label) {
+      return new Item(this.ns, this.key, this.type, this.value, this.shortValue, this.url, label);
+    }
+
+    private Item withUrl(String url) {
+      return new Item(this.ns, this.key, this.type, this.value, this.shortValue, url, this.label);
+    }
+  }
+
+  /**
+   * Unique identifier for a display metadata item within a component.
+   * Identifiers are composed of the key they are registered with and a namespace generated from
+   * the class of the component which registered the item.
+   *
+   * <p>Display metadata registered with the same key from different components will have different
+   * namespaces and thus will both be represented in the composed {@link DisplayData}. If a
+   * single component registers multiple metadata items with the same key, only the most recent
+   * item will be retained; previous versions are discarded.
+   */
+  public static class Identifier {
+    private final String ns;
+    private final String key;
+
+    static Identifier of(Class<?> namespace, String key) {
+      return new Identifier(namespace.getName(), key);
+    }
+
+    private Identifier(String ns, String key) {
+      this.ns = ns;
+      this.key = key;
+    }
+
+    public String getNamespace() {
+      return ns;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof Identifier) {
+        Identifier that = (Identifier) obj;
+        return Objects.equals(this.ns, that.ns)
+          && Objects.equals(this.key, that.key);
+      }
+
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(ns, key);
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%s:%s", ns, key);
+    }
+  }
+
+  /**
+   * Display metadata type.
+   */
+  enum Type {
+    STRING {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue((String) value);
+      }
+    },
+    INTEGER {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue(Long.toString((long) value));
+      }
+    },
+    FLOAT {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue(Double.toString((Double) value));
+      }
+    },
+    TIMESTAMP() {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue((TIMESTAMP_FORMATTER.print((Instant) value)));
+      }
+    },
+    DURATION {
+      @Override
+      FormattedItemValue format(Object value) {
+        return new FormattedItemValue(Long.toString(((Duration) value).getMillis()));
+      }
+    },
+    JAVA_CLASS {
+      @Override
+      FormattedItemValue format(Object value) {
+        Class<?> clazz = (Class<?>) value;
+        return new FormattedItemValue(clazz.getName(), clazz.getSimpleName());
+      }
+    };
+
+    /**
+     * Format the display metadata value into a long string representation, and optionally
+     * a shorter representation for display.
+     *
+     * <p>Internal-only. Value objects can be safely cast to the expected Java type.
+     */
+    abstract FormattedItemValue format(Object value);
+  }
+
+  private static class FormattedItemValue {
+    private final String shortValue;
+    private final String longValue;
+
+    private FormattedItemValue(String longValue) {
+      this(longValue, null);
+    }
+
+    private FormattedItemValue(String longValue, String shortValue) {
+      this.longValue = longValue;
+      this.shortValue = shortValue;
+    }
+
+    private String getLongValue () {
+      return this.longValue;
+    }
+
+    private String getShortValue() {
+      return this.shortValue;
+    }
+  }
+
+  private static class InternalBuilder implements ItemBuilder {
+    private final Map<Identifier, Item> entries;
+    private final Set<Object> visited;
+
+    private Class<?> latestNs;
+    private Item latestItem;
+    private Identifier latestIdentifier;
+
+    private InternalBuilder() {
+      this.entries = Maps.newHashMap();
+      this.visited = Sets.newIdentityHashSet();
+    }
+
+    private static InternalBuilder forRoot(HasDisplayData instance) {
+      InternalBuilder builder = new InternalBuilder();
+      builder.include(instance);
+      return builder;
+    }
+
+    @Override
+    public Builder include(HasDisplayData subComponent) {
+      checkNotNull(subComponent);
+      boolean newComponent = visited.add(subComponent);
+      if (newComponent) {
+        Class prevNs = this.latestNs;
+        this.latestNs = subComponent.getClass();
+        subComponent.populateDisplayData(this);
+        this.latestNs = prevNs;
+      }
+
+      return this;
+    }
+
+    @Override
+    public ItemBuilder add(String key, String value) {
+      checkNotNull(value);
+      return addItem(key, Type.STRING, value);
+    }
+
+    @Override
+    public ItemBuilder add(String key, long value) {
+      return addItem(key, Type.INTEGER, value);
+    }
+
+    @Override
+    public ItemBuilder add(String key, double value) {
+      return addItem(key, Type.FLOAT, value);
+    }
+
+    @Override
+    public ItemBuilder add(String key, Instant value) {
+      checkNotNull(value);
+      return addItem(key, Type.TIMESTAMP, value);
+    }
+
+    @Override
+    public ItemBuilder add(String key, Duration value) {
+      checkNotNull(value);
+      return addItem(key, Type.DURATION, value);
+    }
+
+    @Override
+    public ItemBuilder add(String key, Class<?> value) {
+      checkNotNull(value);
+      return addItem(key, Type.JAVA_CLASS, value);
+    }
+
+    private <T> ItemBuilder addItem(String key, Type type, T value) {
+      checkNotNull(key);
+      checkArgument(!key.isEmpty());
+
+      Identifier id = Identifier.of(latestNs, key);
+      if (entries.containsKey(id)) {
+        throw new IllegalArgumentException("DisplayData key already exists. All display data "
+          + "for a component must be registered with a unique key.\nKey: " + id);
+      }
+      Item item = Item.create(id.getNamespace(), key, type, value);
+      entries.put(id, item);
+
+      latestItem = item;
+      latestIdentifier = id;
+
+      return this;
+    }
+
+    @Override
+    public ItemBuilder withLabel(String label) {
+      latestItem = latestItem.withLabel(label);
+      entries.put(latestIdentifier, latestItem);
+      return this;
+    }
+
+    @Override
+    public ItemBuilder withLinkUrl(String url) {
+      latestItem = latestItem.withUrl(url);
+      entries.put(latestIdentifier, latestItem);
+      return this;
+    }
+
+    private DisplayData build() {
+      return new DisplayData(this.entries);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java
new file mode 100644
index 0000000..b2eca3d
--- /dev/null
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright (C) 2016 Google Inc.
+ *
+ * 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.google.cloud.dataflow.sdk.transforms.display;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+
+/**
+ * Marker interface for {@link PTransform PTransforms} and components used within
+ * {@link PTransform PTransforms} to specify display metadata to be used within UIs and diagnostic
+ * tools.
+ *
+ * <p>Display metadata is optional and may be collected during pipeline construction. It should
+ * only be used to informational purposes. Tools and components should not assume that display data
+ * will always be collected, or that collected display data will always be displayed.
+ */
+public interface HasDisplayData {
+  /**
+   * Register display metadata for the given transform or component. Metadata can be registered
+   * directly on the provided builder, as well as via included sub-components.
+   *
+   * <pre>
+   * {@code
+   * @Override
+   * public void populateDisplayData(DisplayData.Builder builder) {
+   *  builder
+   *     .include(subComponent)
+   *     .add("minFilter", 42)
+   *     .add("topic", "projects/myproject/topics/mytopic")
+   *       .withLabel("Pub/Sub Topic")
+   *     .add("serviceInstance", "myservice.com/fizzbang")
+   *       .withLinkUrl("http://www.myservice.com/fizzbang");
+   * }
+   * }
+   * </pre>
+   *
+   * @param builder The builder to populate with display metadata.
+   */
+  void populateDisplayData(DisplayData.Builder builder);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTest.java
index a709a23..dabad7b 100644
--- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTest.java
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnTest.java
@@ -17,14 +17,17 @@
 package com.google.cloud.dataflow.sdk.transforms;
 
 import static org.hamcrest.CoreMatchers.isA;
+import static org.hamcrest.Matchers.empty;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertThat;
 
 import com.google.cloud.dataflow.sdk.Pipeline.PipelineExecutionException;
 import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
 import com.google.cloud.dataflow.sdk.testing.TestPipeline;
 import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn;
 import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -188,4 +191,16 @@ public class DoFnTest implements Serializable {
 
     return pipeline;
   }
+
+  @Test
+  public void testPopulateDisplayDataDefaultBehavior() {
+    DoFn<String, String> usesDefault =
+        new DoFn<String, String>() {
+          @Override
+          public void processElement(ProcessContext c) throws Exception {}
+        };
+
+    DisplayData data = DisplayData.from(usesDefault);
+    assertThat(data.items(), empty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PTransformTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PTransformTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PTransformTest.java
new file mode 100644
index 0000000..cea1b38
--- /dev/null
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PTransformTest.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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.google.cloud.dataflow.sdk.transforms;
+
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertThat;
+
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link PTransform} base class.
+ */
+@RunWith(JUnit4.class)
+public class PTransformTest {
+  @Test
+  public void testPopulateDisplayDataDefaultBehavior() {
+    PTransform<PCollection<String>, PCollection<String>> transform =
+        new PTransform<PCollection<String>, PCollection<String>>() {};
+    DisplayData displayData = DisplayData.from(transform);
+    assertThat(displayData.items(), empty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java
index f3f9bde..1ff46e4 100644
--- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java
@@ -16,6 +16,8 @@
 
 package com.google.cloud.dataflow.sdk.transforms;
 
+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.util.SerializableUtils.serializeToByteArray;
 import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString;
 import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray;
@@ -39,6 +41,9 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
 import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
 import com.google.cloud.dataflow.sdk.testing.TestPipeline;
 import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
+import com.google.cloud.dataflow.sdk.transforms.ParDo.Bound;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
 import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
 import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
 import com.google.cloud.dataflow.sdk.util.IllegalMutationException;
@@ -1515,4 +1520,22 @@ public class ParDoTest implements Serializable {
     thrown.expectMessage("must not be mutated");
     pipeline.run();
   }
+
+  @Test
+  public void testIncludesDoFnDisplayData() {
+    Bound<String, String> parDo =
+        ParDo.of(
+            new DoFn<String, String>() {
+              @Override
+              public void processElement(ProcessContext c) {}
+
+              @Override
+              public void populateDisplayData(Builder builder) {
+                builder.add("foo", "bar");
+              }
+            });
+
+    DisplayData displayData = DisplayData.from(parDo);
+    assertThat(displayData, hasDisplayItem(hasKey("foo")));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
new file mode 100644
index 0000000..2753aaf
--- /dev/null
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright (C) 2016 Google Inc.
+ *
+ * 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.google.cloud.dataflow.sdk.transforms.display;
+
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
+
+import org.hamcrest.Description;
+import org.hamcrest.FeatureMatcher;
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeDiagnosingMatcher;
+
+import java.util.Collection;
+
+/**
+ * Hamcrest matcher for making assertions on {@link DisplayData} instances.
+ */
+public class DisplayDataMatchers {
+  /**
+   * Do not instantiate.
+   */
+  private DisplayDataMatchers() {}
+
+  /**
+   * Creates a matcher that matches if the examined {@link DisplayData} contains any items.
+   */
+  public static Matcher<DisplayData> hasDisplayItem() {
+    return hasDisplayItem(Matchers.any(DisplayData.Item.class));
+  }
+
+  /**
+   * Creates a matcher that matches if the examined {@link DisplayData} contains any item
+   * matching the specified {@code itemMatcher}.
+   */
+  public static Matcher<DisplayData> hasDisplayItem(Matcher<DisplayData.Item> itemMatcher) {
+    return new HasDisplayDataItemMatcher(itemMatcher);
+  }
+
+  private static class HasDisplayDataItemMatcher extends TypeSafeDiagnosingMatcher<DisplayData> {
+    private final Matcher<Item> itemMatcher;
+
+    private HasDisplayDataItemMatcher(Matcher<DisplayData.Item> itemMatcher) {
+      this.itemMatcher = itemMatcher;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("display data with item: ");
+      itemMatcher.describeTo(description);
+    }
+
+    @Override
+    protected boolean matchesSafely(DisplayData data, Description mismatchDescription) {
+      Collection<Item> items = data.items();
+      boolean isMatch = Matchers.hasItem(itemMatcher).matches(items);
+      if (!isMatch) {
+        mismatchDescription.appendText("found " + items.size() + " non-matching items");
+      }
+
+      return isMatch;
+    }
+  }
+
+  /**
+   * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key
+   * with the specified value.
+   */
+  public static Matcher<DisplayData.Item> hasKey(String key) {
+    return hasKey(Matchers.is(key));
+  }
+
+  /**
+   * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key
+   * matching the specified key matcher.
+   */
+  public static Matcher<DisplayData.Item> hasKey(Matcher<String> keyMatcher) {
+    return new FeatureMatcher<DisplayData.Item, String>(keyMatcher, "with key", "key") {
+      @Override
+      protected String featureValueOf(DisplayData.Item actual) {
+        return actual.getKey();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
new file mode 100644
index 0000000..2636cf8
--- /dev/null
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright (C) 2016 Google Inc.
+ *
+ * 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.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 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;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.StringDescription;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Unit tests for {@link DisplayDataMatchers}.
+ */
+@RunWith(JUnit4.class)
+public class DisplayDataMatchersTest {
+  @Test
+  public void testHasDisplayItem() {
+    Matcher<DisplayData> matcher = hasDisplayItem();
+
+    assertFalse(matcher.matches(DisplayData.none()));
+    assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar")));
+  }
+
+  @Test
+  public void testHasDisplayItemDescription() {
+    Matcher<DisplayData> matcher = hasDisplayItem();
+    Description desc = new StringDescription();
+    Description mismatchDesc = new StringDescription();
+
+    matcher.describeTo(desc);
+    matcher.describeMismatch(DisplayData.none(), mismatchDesc);
+
+    assertThat(desc.toString(), startsWith("display data with item: "));
+    assertThat(mismatchDesc.toString(), containsString("found 0 non-matching items"));
+  }
+
+  @Test
+  public void testHasKey() {
+    Matcher<DisplayData> matcher = hasDisplayItem(hasKey("foo"));
+
+    assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar")));
+    assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar")));
+  }
+
+  private DisplayData createDisplayDataWithItem(final String key, final String value) {
+    return DisplayData.from(
+        new PTransform<PCollection<String>, PCollection<String>>() {
+          @Override
+          public void populateDisplayData(Builder builder) {
+            builder.add(key, value);
+          }
+        });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
new file mode 100644
index 0000000..13dd618
--- /dev/null
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java
@@ -0,0 +1,633 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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.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 org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isEmptyOrNullString;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder;
+import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.testing.EqualsTester;
+
+import org.hamcrest.CustomTypeSafeMatcher;
+import org.hamcrest.FeatureMatcher;
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Tests for {@link DisplayData} class.
+ */
+@RunWith(JUnit4.class)
+public class DisplayDataTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+  private static final DateTimeFormatter ISO_FORMATTER = ISODateTimeFormat.dateTime();
+
+  @Test
+  public void testTypicalUsage() {
+    final HasDisplayData subComponent1 =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder.add("ExpectedAnswer", 42);
+          }
+        };
+
+    final HasDisplayData subComponent2 =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder.add("Location", "Seattle").add("Forecast", "Rain");
+          }
+        };
+
+    PTransform<?, ?> transform =
+        new PTransform<PCollection<String>, PCollection<String>>() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder
+                .include(subComponent1)
+                .include(subComponent2)
+                .add("MinSproggles", 200)
+                .withLabel("Mimimum Required Sproggles")
+                .add("LazerOrientation", "NORTH")
+                .add("TimeBomb", Instant.now().plus(Duration.standardDays(1)))
+                .add("FilterLogic", subComponent1.getClass())
+                .add("ServiceUrl", "google.com/fizzbang")
+                .withLinkUrl("http://www.google.com/fizzbang");
+          }
+        };
+
+    DisplayData data = DisplayData.from(transform);
+
+    assertThat(data.items(), not(empty()));
+    assertThat(
+        data.items(),
+        everyItem(
+            allOf(
+                hasKey(not(isEmptyOrNullString())),
+                hasNamespace(
+                    Matchers.<Class<?>>isOneOf(
+                        transform.getClass(), subComponent1.getClass(), subComponent2.getClass())),
+                hasType(notNullValue(DisplayData.Type.class)),
+                hasValue(not(isEmptyOrNullString())))));
+  }
+
+  @Test
+  public void testDefaultInstance() {
+    DisplayData none = DisplayData.none();
+    assertThat(none.items(), empty());
+  }
+
+  @Test
+  public void testCanBuild() {
+    DisplayData data =
+        DisplayData.from(new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add("Foo", "bar");
+              }
+            });
+
+    assertThat(data.items(), hasSize(1));
+    assertThat(data, hasDisplayItem(hasKey("Foo")));
+  }
+
+  @Test
+  public void testAsMap() {
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add("foo", "bar");
+              }
+            });
+
+    Map<DisplayData.Identifier, DisplayData.Item> map = data.asMap();
+    assertEquals(map.size(), 1);
+    assertThat(data, hasDisplayItem(hasKey("foo")));
+    assertEquals(map.values(), data.items());
+  }
+
+  @Test
+  public void testItemProperties() {
+    final Instant value = Instant.now();
+    DisplayData data = DisplayData.from(new ConcreteComponent(value));
+
+    @SuppressWarnings("unchecked")
+    DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0];
+    assertThat(
+        item,
+        allOf(
+            hasNamespace(Matchers.<Class<?>>is(ConcreteComponent.class)),
+            hasKey("now"),
+            hasType(is(DisplayData.Type.TIMESTAMP)),
+            hasValue(is(ISO_FORMATTER.print(value))),
+            hasShortValue(nullValue(String.class)),
+            hasLabel(is("the current instant")),
+            hasUrl(is("http://time.gov"))));
+  }
+
+  static class ConcreteComponent implements HasDisplayData {
+    private Instant value;
+
+    ConcreteComponent(Instant value) {
+      this.value = value;
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      builder.add("now", value).withLabel("the current instant").withLinkUrl("http://time.gov");
+    }
+  }
+
+  @Test
+  public void testUnspecifiedOptionalProperties() {
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add("foo", "bar");
+              }
+            });
+
+    assertThat(
+        data,
+        hasDisplayItem(allOf(hasLabel(nullValue(String.class)), hasUrl(nullValue(String.class)))));
+  }
+
+  @Test
+  public void testIncludes() {
+    final HasDisplayData subComponent =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder.add("foo", "bar");
+          }
+        };
+
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.include(subComponent);
+              }
+            });
+
+    assertThat(
+        data,
+        hasDisplayItem(
+            allOf(
+                hasKey("foo"),
+                hasNamespace(Matchers.<Class<?>>is(subComponent.getClass())))));
+  }
+
+  @Test
+  public void testIdentifierEquality() {
+    new EqualsTester()
+        .addEqualityGroup(
+            DisplayData.Identifier.of(DisplayDataTest.class, "1"),
+            DisplayData.Identifier.of(DisplayDataTest.class, "1"))
+        .addEqualityGroup(DisplayData.Identifier.of(Object.class, "1"))
+        .addEqualityGroup(DisplayData.Identifier.of(DisplayDataTest.class, "2"))
+        .testEquals();
+  }
+
+  @Test
+  public void testAnonymousClassNamespace() {
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder.add("foo", "bar");
+              }
+            });
+
+    DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0];
+    final Pattern anonClassRegex = Pattern.compile(
+        Pattern.quote(DisplayDataTest.class.getName()) + "\\$\\d+$");
+    assertThat(item.getNamespace(), new CustomTypeSafeMatcher<String>(
+        "anonymous class regex: " + anonClassRegex) {
+      @Override
+      protected boolean matchesSafely(String item) {
+        java.util.regex.Matcher m = anonClassRegex.matcher(item);
+        return m.matches();
+      }
+    });
+  }
+
+  @Test
+  public void testAcceptsKeysWithDifferentNamespaces() {
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder
+                    .add("foo", "bar")
+                    .include(
+                        new HasDisplayData() {
+                          @Override
+                          public void populateDisplayData(DisplayData.Builder builder) {
+                            builder.add("foo", "bar");
+                          }
+                        });
+              }
+            });
+
+    assertThat(data.items(), hasSize(2));
+  }
+
+  @Test
+  public void testDuplicateKeyThrowsException() {
+    thrown.expect(IllegalArgumentException.class);
+    DisplayData.from(
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder
+              .add("foo", "bar")
+              .add("foo", "baz");
+          }
+        });
+  }
+
+  @Test
+  public void testToString() {
+    HasDisplayData component = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder.add("foo", "bar");
+      }
+    };
+
+    DisplayData data = DisplayData.from(component);
+    assertEquals(String.format("%s:foo=bar", component.getClass().getName()), data.toString());
+  }
+
+  @Test
+  public void testHandlesIncludeCycles() {
+
+    final IncludeSubComponent componentA =
+        new IncludeSubComponent() {
+          @Override
+          String getId() {
+            return "componentA";
+          }
+        };
+    final IncludeSubComponent componentB =
+        new IncludeSubComponent() {
+          @Override
+          String getId() {
+            return "componentB";
+          }
+        };
+
+    HasDisplayData component =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(Builder builder) {
+            builder.include(componentA);
+          }
+        };
+
+    componentA.subComponent = componentB;
+    componentB.subComponent = componentA;
+
+    DisplayData data = DisplayData.from(component);
+    assertThat(data.items(), hasSize(2));
+  }
+
+  @Test
+  public void testIncludesSubcomponentsWithObjectEquality() {
+    DisplayData data = DisplayData.from(new HasDisplayData() {
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder
+          .include(new EqualsEverything("foo1", "bar1"))
+          .include(new EqualsEverything("foo2", "bar2"));
+      }
+    });
+
+    assertThat(data.items(), hasSize(2));
+  }
+
+  private static class EqualsEverything implements HasDisplayData {
+    private final String value;
+    private final String key;
+    EqualsEverything(String key, String value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      builder.add(key, value);
+    }
+
+    @Override
+    public int hashCode() {
+      return 1;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return true;
+    }
+  }
+
+  abstract static class IncludeSubComponent implements HasDisplayData {
+    HasDisplayData subComponent;
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      builder.add("id", getId()).include(subComponent);
+    }
+
+    abstract String getId();
+  }
+
+  @Test
+  public void testTypeMappings() {
+    DisplayData data =
+        DisplayData.from(
+            new HasDisplayData() {
+              @Override
+              public void populateDisplayData(DisplayData.Builder builder) {
+                builder
+                    .add("string", "foobar")
+                    .add("integer", 123)
+                    .add("float", 3.14)
+                    .add("java_class", DisplayDataTest.class)
+                    .add("timestamp", Instant.now())
+                    .add("duration", Duration.standardHours(1));
+              }
+            });
+
+    Collection<Item> items = data.items();
+    assertThat(
+        items, hasItem(allOf(hasKey("string"), hasType(is(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)))));
+    assertThat(
+        items,
+        hasItem(allOf(hasKey("java_class"), hasType(is(DisplayData.Type.JAVA_CLASS)))));
+    assertThat(
+        items,
+        hasItem(allOf(hasKey("timestamp"), hasType(is(DisplayData.Type.TIMESTAMP)))));
+    assertThat(
+        items, hasItem(allOf(hasKey("duration"), hasType(is(DisplayData.Type.DURATION)))));
+  }
+
+  @Test
+  public void testStringFormatting() throws IOException {
+    final Instant now = Instant.now();
+    final Duration oneHour = Duration.standardHours(1);
+
+    HasDisplayData component = new HasDisplayData() {
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder
+          .add("string", "foobar")
+          .add("integer", 123)
+          .add("float", 3.14)
+          .add("java_class", DisplayDataTest.class)
+          .add("timestamp", now)
+          .add("duration", oneHour);
+      }
+    };
+    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()))))));
+  }
+
+  @Test
+  public void testContextProperlyReset() {
+    final HasDisplayData subComponent =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder.add("foo", "bar");
+          }
+        };
+
+    HasDisplayData component =
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(DisplayData.Builder builder) {
+            builder
+              .include(subComponent)
+              .add("alpha", "bravo");
+          }
+        };
+
+    DisplayData data = DisplayData.from(component);
+    assertThat(
+        data.items(),
+        hasItem(
+            allOf(
+                hasKey("alpha"),
+                hasNamespace(Matchers.<Class<?>>is(component.getClass())))));
+  }
+
+  @Test
+  public void testFromNull() {
+    thrown.expect(NullPointerException.class);
+    DisplayData.from(null);
+  }
+
+  @Test
+  public void testIncludeNull() {
+    thrown.expect(NullPointerException.class);
+    DisplayData.from(
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(Builder builder) {
+            builder.include(null);
+          }
+        });
+  }
+
+  @Test
+  public void testNullKey() {
+    thrown.expect(NullPointerException.class);
+    DisplayData.from(
+        new HasDisplayData() {
+          @Override
+          public void populateDisplayData(Builder builder) {
+            builder.add(null, "foo");
+          }
+        });
+  }
+
+  @Test
+  public void testRejectsNullValues() {
+    DisplayData.from(
+      new HasDisplayData() {
+        @Override
+        public void populateDisplayData(Builder builder) {
+          try {
+            builder.add("key", (String) null);
+            throw new RuntimeException("Should throw on null string value");
+          } catch (NullPointerException ex) {
+            // Expected
+          }
+
+          try {
+            builder.add("key", (Class<?>) null);
+            throw new RuntimeException("Should throw on null class value");
+          } catch (NullPointerException ex) {
+            // Expected
+          }
+
+          try {
+            builder.add("key", (Duration) null);
+            throw new RuntimeException("Should throw on null duration value");
+          } catch (NullPointerException ex) {
+            // Expected
+          }
+
+          try {
+            builder.add("key", (Instant) null);
+            throw new RuntimeException("Should throw on null instant value");
+          } catch (NullPointerException ex) {
+            // Expected
+          }
+        }
+      });
+  }
+
+  public void testAcceptsNullOptionalValues() {
+    DisplayData.from(
+      new HasDisplayData() {
+        @Override
+        public void populateDisplayData(Builder builder) {
+          builder.add("key", "value")
+                  .withLabel(null)
+                  .withLinkUrl(null);
+        }
+      });
+
+    // Should not throw
+  }
+
+  private static Matcher<DisplayData.Item> hasNamespace(Matcher<Class<?>> nsMatcher) {
+    return new FeatureMatcher<DisplayData.Item, Class<?>>(
+        nsMatcher, "display item with namespace", "namespace") {
+      @Override
+      protected Class<?> featureValueOf(DisplayData.Item actual) {
+        try {
+          return Class.forName(actual.getNamespace());
+        } catch (ClassNotFoundException e) {
+          return null;
+        }
+      }
+    };
+  }
+
+  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") {
+      @Override
+      protected String featureValueOf(DisplayData.Item actual) {
+        return actual.getLabel();
+      }
+    };
+  }
+
+  private static Matcher<DisplayData.Item> hasUrl(Matcher<String> urlMatcher) {
+    return new FeatureMatcher<DisplayData.Item, String>(
+        urlMatcher, "display item with url", "URL") {
+      @Override
+      protected String featureValueOf(DisplayData.Item actual) {
+        return actual.getUrl();
+      }
+    };
+  }
+
+  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") {
+      @Override
+      protected String featureValueOf(DisplayData.Item actual) {
+        return actual.getShortValue();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5ecb7aa7/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java
index e995b82..fcfe1d8 100644
--- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java
+++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ApiSurfaceTest.java
@@ -49,7 +49,8 @@ public class ApiSurfaceTest {
       .pruningClassName("com.google.cloud.dataflow.sdk.util.common.ReflectHelpers")
       .pruningClassName("com.google.cloud.dataflow.sdk.DataflowMatchers")
       .pruningClassName("com.google.cloud.dataflow.sdk.TestUtils")
-      .pruningClassName("com.google.cloud.dataflow.sdk.WindowMatchers");
+      .pruningClassName("com.google.cloud.dataflow.sdk.WindowMatchers")
+      .pruningClassName("com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers");
 
     checkedApiSurface.getExposedClasses();