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 <sweg...@google.com> Authored: Thu Mar 17 10:22:42 2016 -0700 Committer: Maximilian Michels <m...@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();