Repository: incubator-beam Updated Branches: refs/heads/master ff6301bd6 -> 5047cf746
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java index 5a58519..4d7814c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.io; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -74,7 +74,7 @@ public class BoundedReadFromUnboundedSourceTest implements Serializable{ }; BoundedReadFromUnboundedSource<KV<Integer, Integer>> read = Read.from(src).withMaxNumRecords(5); - assertThat(DisplayData.from(read), includesDisplayDataFrom(src)); + assertThat(DisplayData.from(read), includesDisplayDataFor("source", src)); } private static class Checker http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index d7c451d..f8769ea 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; @@ -424,7 +424,7 @@ public class CompressedSourceTest { assertThat(compressedSourceDisplayData, hasDisplayItem("compressionMode")); assertThat(gzipDisplayData, hasDisplayItem("compressionMode", CompressionMode.GZIP.toString())); assertThat(compressedSourceDisplayData, hasDisplayItem("source", inputSource.getClass())); - assertThat(compressedSourceDisplayData, includesDisplayDataFrom(inputSource)); + assertThat(compressedSourceDisplayData, includesDisplayDataFor("source", inputSource)); } /** http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index a5138c5..2e90f9a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasItem; @@ -96,11 +96,11 @@ public class ReadTest implements Serializable{ DisplayData boundedDisplayData = DisplayData.from(bounded); assertThat(boundedDisplayData, hasDisplayItem("source", boundedSource.getClass())); - assertThat(boundedDisplayData, includesDisplayDataFrom(boundedSource)); + assertThat(boundedDisplayData, includesDisplayDataFor("source", boundedSource)); DisplayData unboundedDisplayData = DisplayData.from(unbounded); assertThat(unboundedDisplayData, hasDisplayItem("source", unboundedSource.getClass())); - assertThat(unboundedDisplayData, includesDisplayDataFrom(unboundedSource)); + assertThat(unboundedDisplayData, includesDisplayDataFor("source", unboundedSource)); assertThat(unboundedDisplayData, hasDisplayItem("maxRecords", 1234)); assertThat(unboundedDisplayData, hasDisplayItem("maxReadTime", maxReadTime)); } @@ -142,12 +142,12 @@ public class ReadTest implements Serializable{ Set<DisplayData> boundedDisplayData = evaluator .displayDataForPrimitiveSourceTransforms(bounded); assertThat(boundedDisplayData, hasItem(hasDisplayItem("source", boundedSource.getClass()))); - assertThat(boundedDisplayData, hasItem(includesDisplayDataFrom(boundedSource))); + assertThat(boundedDisplayData, hasItem(includesDisplayDataFor("source", boundedSource))); Set<DisplayData> unboundedDisplayData = evaluator .displayDataForPrimitiveSourceTransforms(unbounded); assertThat(unboundedDisplayData, hasItem(hasDisplayItem("source"))); - assertThat(unboundedDisplayData, hasItem(includesDisplayDataFrom(unboundedSource))); + assertThat(unboundedDisplayData, hasItem(includesDisplayDataFor("source", unboundedSource))); } private abstract static class CustomBoundedSource extends BoundedSource<String> { http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index 997566a..5be5ff1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -274,7 +274,7 @@ public class WriteTest { DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); - assertThat(displayData, includesDisplayDataFrom(sink)); + assertThat(displayData, includesDisplayDataFor("sink", sink)); } @Test @@ -288,7 +288,7 @@ public class WriteTest { Write.Bound<String> write = Write.to(sink).withNumShards(1); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); - assertThat(displayData, includesDisplayDataFrom(sink)); + assertThat(displayData, includesDisplayDataFor("sink", sink)); assertThat(displayData, hasDisplayItem("numShards", 1)); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java index 32222d3..52b98ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java @@ -24,6 +24,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -34,6 +35,7 @@ import static org.junit.Assert.assertTrue; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -48,11 +50,17 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.display.DisplayData; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.rules.ExpectedException; import org.junit.rules.ExternalResource; import org.junit.rules.TestRule; @@ -756,6 +764,38 @@ public class ProxyInvocationHandlerTest { } @Test + @Category(NeedsRunner.class) + public void pipelineOptionsDisplayDataExceptionShouldFail() { + Object brokenValueType = new Object() { + @JsonValue + public int getValue () { + return 42; + } + + @Override + public String toString() { + throw new RuntimeException("oh noes!!"); + } + }; + + Pipeline p = TestPipeline.create(); + p.getOptions().as(ObjectPipelineOptions.class).setValue(brokenValueType); + + p.apply(Create.of(1, 2, 3)); + + expectedException.expectMessage( + ProxyInvocationHandler.PipelineOptionsDisplayData.class.getName()); + expectedException.expectCause(ThrowableMessageMatcher.hasMessage(is("oh noes!!"))); + p.run(); + } + + /** {@link PipelineOptions} to inject bad object implementations. */ + public interface ObjectPipelineOptions extends PipelineOptions { + Object getValue(); + void setValue(Object value); + } + + @Test public void testDisplayDataInheritanceNamespace() { ExtendsBaseOptions options = PipelineOptionsFactory.as(ExtendsBaseOptions.class); options.setFoo("bar"); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java index 35f9858..8862531 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; @@ -298,9 +298,8 @@ public class CombineFnsTest { assertThat(displayData, hasDisplayItem("combineFn1", combineFn1.getClass())); assertThat(displayData, hasDisplayItem("combineFn2", combineFn2.getClass())); - String nsBase = DisplayDataCombineFn.class.getName(); - assertThat(displayData, includesDisplayDataFrom(combineFn1, nsBase + "#1")); - assertThat(displayData, includesDisplayDataFrom(combineFn2, nsBase + "#2")); + assertThat(displayData, includesDisplayDataFor("combineFn1", combineFn1)); + assertThat(displayData, includesDisplayDataFor("combineFn2", combineFn2)); } private static class DisplayDataCombineFn extends Combine.CombineFn<String, String, String> { http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 5ce8055..671f00e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -23,7 +23,7 @@ import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -682,7 +682,7 @@ public class CombineTest implements Serializable { assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); assertThat(displayData, hasDisplayItem("emitDefaultOnEmptyInput", true)); assertThat(displayData, hasDisplayItem("fanout", 1234)); - assertThat(displayData, includesDisplayDataFrom(combineFn)); + assertThat(displayData, includesDisplayDataFor("combineFn", combineFn)); } @Test http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index bda696f..52244a0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -21,7 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; @@ -1432,7 +1432,7 @@ public class ParDoTest implements Serializable { hasType(DisplayData.Type.JAVA_CLASS), DisplayDataMatchers.hasValue(fn.getClass().getName())))); - assertThat(displayData, includesDisplayDataFrom(fn)); + assertThat(displayData, includesDisplayDataFor("fn", fn)); } @Test @@ -1450,7 +1450,7 @@ public class ParDoTest implements Serializable { Bound<String, String> parDo = ParDo.of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includesDisplayDataFrom(fn)); + assertThat(displayData, includesDisplayDataFor("fn", fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } @@ -1494,7 +1494,7 @@ public class ParDoTest implements Serializable { .of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includesDisplayDataFrom(fn)); + assertThat(displayData, includesDisplayDataFor("fn", fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java index e9db522..7e0bd12 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java @@ -18,8 +18,10 @@ package org.apache.beam.sdk.transforms.display; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.is; import com.google.common.collect.Sets; +import java.util.Arrays; import java.util.Collection; import org.apache.beam.sdk.transforms.display.DisplayData.Item; import org.hamcrest.CustomTypeSafeMatcher; @@ -44,10 +46,10 @@ public class DisplayDataMatchers { * Creates a matcher that matches if the examined {@link DisplayData} contains any items. */ public static Matcher<DisplayData> hasDisplayItem() { - return new FeatureMatcher<DisplayData, Collection<DisplayData.Item<?>>>( + return new FeatureMatcher<DisplayData, Collection<DisplayData.Item>>( Matchers.not(Matchers.empty()), "DisplayData", "DisplayData") { @Override - protected Collection<Item<?>> featureValueOf(DisplayData actual) { + protected Collection<Item> featureValueOf(DisplayData actual) { return actual.items(); } }; @@ -130,14 +132,14 @@ public class DisplayDataMatchers { * 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) { + 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 final Matcher<Item> itemMatcher; - private HasDisplayDataItemMatcher(Matcher<DisplayData.Item<?>> itemMatcher) { + private HasDisplayDataItemMatcher(Matcher<DisplayData.Item> itemMatcher) { this.itemMatcher = itemMatcher; } @@ -149,7 +151,7 @@ public class DisplayDataMatchers { @Override protected boolean matchesSafely(DisplayData data, Description mismatchDescription) { - Collection<Item<?>> items = data.items(); + Collection<Item> items = data.items(); boolean isMatch = Matchers.hasItem(itemMatcher).matches(items); if (!isMatch) { mismatchDescription.appendText("found " + items.size() + " non-matching item(s):\n"); @@ -160,42 +162,31 @@ public class DisplayDataMatchers { } } - /** @see #includesDisplayDataFrom(HasDisplayData, String) */ - public static Matcher<DisplayData> includesDisplayDataFrom(HasDisplayData subComponent) { - return includesDisplayDataFrom(subComponent, subComponent.getClass()); - } - - /** @see #includesDisplayDataFrom(HasDisplayData, String) */ - public static Matcher<DisplayData> includesDisplayDataFrom( - HasDisplayData subComponent, Class<? extends HasDisplayData> namespace) { - return includesDisplayDataFrom(subComponent, namespace.getName()); - } - /** * Create a matcher that matches if the examined {@link DisplayData} contains all display data * registered from the specified subcomponent and namespace. */ - public static Matcher<DisplayData> includesDisplayDataFrom( - final HasDisplayData subComponent, final String namespace) { + public static Matcher<DisplayData> includesDisplayDataFor( + final String path, final HasDisplayData subComponent) { return new CustomTypeSafeMatcher<DisplayData>("includes subcomponent") { @Override protected boolean matchesSafely(DisplayData displayData) { - DisplayData subComponentData = subComponentData(); + DisplayData subComponentData = subComponentData(path); if (subComponentData.items().size() == 0) { throw new UnsupportedOperationException("subComponent contains no display data; " + "cannot verify whether it is included"); } - DisplayDataComparison comparison = checkSubset(displayData, subComponentData); + DisplayDataComparison comparison = checkSubset(displayData, subComponentData, path); return comparison.missingItems.isEmpty(); } @Override protected void describeMismatchSafely( DisplayData displayData, Description mismatchDescription) { - DisplayData subComponentDisplayData = subComponentData(); + DisplayData subComponentDisplayData = subComponentData(path); DisplayDataComparison comparison = checkSubset( - displayData, subComponentDisplayData); + displayData, subComponentDisplayData, path); mismatchDescription .appendText("did not include:\n") @@ -204,21 +195,21 @@ public class DisplayDataMatchers { .appendValue(comparison.unmatchedItems); } - private DisplayData subComponentData() { + private DisplayData subComponentData(final String path) { return DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.include(subComponent, namespace); + builder.include(path, subComponent); } }); } private DisplayDataComparison checkSubset( - DisplayData displayData, DisplayData included) { + DisplayData displayData, DisplayData included, String path) { DisplayDataComparison comparison = new DisplayDataComparison(displayData.items()); - for (Item<?> item : included.items()) { - Item<?> matchedItem = displayData.asMap().get( - DisplayData.Identifier.of(item.getNamespace(), item.getKey())); + for (Item item : included.items()) { + Item matchedItem = displayData.asMap().get(DisplayData.Identifier.of( + DisplayData.Path.absolute(path), item.getNamespace(), item.getKey())); if (matchedItem != null) { comparison.matched(matchedItem); @@ -231,19 +222,19 @@ public class DisplayDataMatchers { } class DisplayDataComparison { - Collection<DisplayData.Item<?>> missingItems; - Collection<DisplayData.Item<?>> unmatchedItems; + Collection<Item> missingItems; + Collection<Item> unmatchedItems; - DisplayDataComparison(Collection<Item<?>> superset) { + DisplayDataComparison(Collection<Item> superset) { missingItems = Sets.newHashSet(); unmatchedItems = Sets.newHashSet(superset); } - void matched(Item<?> supersetItem) { + void matched(Item supersetItem) { unmatchedItems.remove(supersetItem); } - void missing(Item<?> subsetItem) { + void missing(Item subsetItem) { missingItems.add(subsetItem); } } @@ -254,59 +245,59 @@ public class DisplayDataMatchers { * 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)); + public static Matcher<DisplayData.Item> hasKey(String key) { + return hasKey(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") { + 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) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getKey(); } }; } + /** - * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the - * specified namespace. + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a path + * matching the specified namespace. */ - public static Matcher<DisplayData.Item<?>> hasNamespace(Class<?> namespace) { - return hasNamespace(Matchers.<Class<?>>is(namespace)); + public static Matcher<DisplayData.Item> hasPath(String... paths) { + DisplayData.Path path = (paths.length == 0) + ? DisplayData.Path.root() + : DisplayData.Path.absolute(paths[0], Arrays.copyOfRange(paths, 1, paths.length)); + return new FeatureMatcher<DisplayData.Item, DisplayData.Path>( + is(path), " with namespace", "namespace") { + @Override + protected DisplayData.Path featureValueOf(DisplayData.Item actual) { + return actual.getPath(); + } + }; } /** * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the * specified namespace. */ - public static Matcher<DisplayData.Item<?>> hasNamespace(String namespace) { - return new FeatureMatcher<DisplayData.Item<?>, String>( - Matchers.is(namespace), "display item with namespace", "namespace") { - @Override - protected String featureValueOf(Item<?> actual) { - return actual.getNamespace(); - } - }; + public static Matcher<DisplayData.Item> hasNamespace(Class<?> namespace) { + return hasNamespace(Matchers.<Class<?>>is(namespace)); } /** * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a namespace * matching the specified namespace matcher. */ - public static Matcher<DisplayData.Item<?>> hasNamespace(Matcher<Class<?>> namespaceMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, Class<?>>( - namespaceMatcher, "display item with namespace", "namespace") { + public static Matcher<DisplayData.Item> hasNamespace(Matcher<Class<?>> namespaceMatcher) { + return new FeatureMatcher<DisplayData.Item, Class<?>>( + namespaceMatcher, " with namespace", "namespace") { @Override - protected Class<?> featureValueOf(DisplayData.Item<?> actual) { - try { - return Class.forName(actual.getNamespace()); - } catch (ClassNotFoundException e) { - return null; - } + protected Class<?> featureValueOf(DisplayData.Item actual) { + return actual.getNamespace(); } }; } @@ -315,19 +306,19 @@ public class DisplayDataMatchers { * Creates a matcher that matches if the examined {@link DisplayData.Item} matches the * specified type. */ - public static Matcher<DisplayData.Item<?>> hasType(DisplayData.Type type) { - return hasType(Matchers.is(type)); + public static Matcher<DisplayData.Item> hasType(DisplayData.Type type) { + return hasType(is(type)); } /** * Creates a matcher that matches if the examined {@link DisplayData.Item} has a type * matching the specified type matcher. */ - public static Matcher<DisplayData.Item<?>> hasType(Matcher<DisplayData.Type> typeMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, DisplayData.Type>( + public static Matcher<DisplayData.Item> hasType(Matcher<DisplayData.Type> typeMatcher) { + return new FeatureMatcher<DisplayData.Item, DisplayData.Type>( typeMatcher, "with type", "type") { @Override - protected DisplayData.Type featureValueOf(DisplayData.Item<?> actual) { + protected DisplayData.Type featureValueOf(DisplayData.Item actual) { return actual.getType(); } }; @@ -338,19 +329,19 @@ public class DisplayDataMatchers { * value. */ - public static Matcher<DisplayData.Item<?>> hasValue(Object value) { - return hasValue(Matchers.is(value)); + public static Matcher<DisplayData.Item> hasValue(Object value) { + return hasValue(is(value)); } /** * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a value * matching the specified value matcher. */ - public static <T> Matcher<DisplayData.Item<?>> hasValue(Matcher<T> valueMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, T>( + public static <T> Matcher<DisplayData.Item> hasValue(Matcher<T> valueMatcher) { + return new FeatureMatcher<DisplayData.Item, T>( valueMatcher, "with value", "value") { @Override - protected T featureValueOf(DisplayData.Item<?> actual) { + protected T featureValueOf(DisplayData.Item actual) { @SuppressWarnings("unchecked") T value = (T) actual.getValue(); return value; @@ -362,19 +353,19 @@ public class DisplayDataMatchers { * Creates a matcher that matches if the examined {@link DisplayData.Item} has the specified * label. */ - public static Matcher<DisplayData.Item<?>> hasLabel(String label) { - return hasLabel(Matchers.is(label)); + public static Matcher<DisplayData.Item> hasLabel(String label) { + return hasLabel(is(label)); } /** * Creates a matcher that matches if the examined {@link DisplayData.Item} has a label matching * the specified label matcher. */ - public static Matcher<DisplayData.Item<?>> hasLabel(Matcher<String> labelMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, String>( + public 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) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getLabel(); } }; http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java index 3ea6830..f7f8d40 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java @@ -19,9 +19,10 @@ package org.apache.beam.sdk.transforms.display; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasPath; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -100,6 +101,32 @@ public class DisplayDataMatchersTest { } @Test + public void testHasPath() { + Matcher<DisplayData> matcher = hasDisplayItem(hasPath("a", "b")); + + final HasDisplayData subComponent = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include("b", new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }); + } + }; + + assertFalse(matcher.matches(DisplayData.from(subComponent))); + + assertThat(DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include("a", subComponent); + } + }), matcher); + } + + @Test public void testHasNamespace() { Matcher<DisplayData> matcher = hasDisplayItem(hasNamespace(SampleTransform.class)); @@ -124,25 +151,47 @@ public class DisplayDataMatchersTest { HasDisplayData hasSubcomponent = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder - .include(subComponent) - .add(DisplayData.item("foo2", "bar2")); + builder.include("p", subComponent); } }; - HasDisplayData sameKeyDifferentNamespace = new HasDisplayData() { + + HasDisplayData wrongPath = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include("q", subComponent); + } + }; + + HasDisplayData deeplyNested = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include("p", new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include("p", subComponent); + } + }); + } + }; + + HasDisplayData sameDisplayItemDifferentComponent = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { builder.add(DisplayData.item("foo", "bar")); } }; - Matcher<DisplayData> matcher = includesDisplayDataFrom(subComponent); - assertFalse(matcher.matches(DisplayData.from(sameKeyDifferentNamespace))); + Matcher<DisplayData> matcher = includesDisplayDataFor("p", subComponent); + + assertFalse("should not match sub-component at different path", + matcher.matches(DisplayData.from(wrongPath))); + assertFalse("should not match deeply nested sub-component", + matcher.matches(DisplayData.from(deeplyNested))); + assertFalse("should not match identical display data from different component", + matcher.matches(DisplayData.from(sameDisplayItemDifferentComponent))); assertThat(DisplayData.from(hasSubcomponent), matcher); - assertThat(DisplayData.from(subComponent), matcher); } - private DisplayData createDisplayDataWithItem(final String key, final String value) { return DisplayData.from(new SampleTransform(key, value)); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index a709bd8..770b836 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -21,13 +21,15 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasLabel; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasPath; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; 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.hasItems; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isA; @@ -51,11 +53,10 @@ import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.regex.Pattern; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.DisplayData.Item; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.FeatureMatcher; @@ -115,8 +116,8 @@ public class DisplayDataTest implements Serializable { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .include(subComponent1) - .include(subComponent2) + .include("p1", subComponent1) + .include("p2", subComponent2) .add(DisplayData.item("minSproggles", 200) .withLabel("Minimum Required Sproggles")) .add(DisplayData.item("fireLasers", true)) @@ -174,7 +175,7 @@ public class DisplayDataTest implements Serializable { } }); - Map<DisplayData.Identifier, DisplayData.Item<?>> map = data.asMap(); + Map<DisplayData.Identifier, DisplayData.Item> map = data.asMap(); assertEquals(map.size(), 1); assertThat(data, hasDisplayItem("foo", "bar")); assertEquals(map.values(), data.items()); @@ -194,10 +195,10 @@ public class DisplayDataTest implements Serializable { }); @SuppressWarnings("unchecked") - DisplayData.Item<?> item = (DisplayData.Item<?>) data.items().toArray()[0]; + DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; @SuppressWarnings("unchecked") - Matcher<Item<?>> matchesAllOf = Matchers.allOf( + Matcher<Item> matchesAllOf = Matchers.allOf( hasNamespace(DisplayDataTest.class), hasKey("now"), hasType(DisplayData.Type.TIMESTAMP), @@ -327,6 +328,82 @@ public class DisplayDataTest implements Serializable { DisplayData.from(component); // should not throw } + @Test + public void testRootPath() { + DisplayData.Path root = DisplayData.Path.root(); + assertThat(root.getComponents(), Matchers.empty()); + } + + @Test + public void testExtendPath() { + DisplayData.Path a = DisplayData.Path.root().extend("a"); + assertThat(a.getComponents(), hasItems("a")); + + DisplayData.Path b = a.extend("b"); + assertThat(b.getComponents(), hasItems("a", "b")); + } + + @Test + public void testExtendNullPathValidation() { + DisplayData.Path root = DisplayData.Path.root(); + thrown.expect(NullPointerException.class); + root.extend(null); + } + + @Test + public void testExtendEmptyPathValidation() { + DisplayData.Path root = DisplayData.Path.root(); + thrown.expect(IllegalArgumentException.class); + root.extend(""); + } + + @Test + public void testAbsolute() { + DisplayData.Path path = DisplayData.Path.absolute("a", "b", "c"); + assertThat(path.getComponents(), hasItems("a", "b", "c")); + } + + @Test + public void testAbsoluteValidationNullFirstPath() { + thrown.expect(NullPointerException.class); + DisplayData.Path.absolute(null, "foo", "bar"); + } + + @Test + public void testAbsoluteValidationEmptyFirstPath() { + thrown.expect(IllegalArgumentException.class); + DisplayData.Path.absolute("", "foo", "bar"); + } + + @Test + public void testAbsoluteValidationNullSubsequentPath() { + thrown.expect(NullPointerException.class); + DisplayData.Path.absolute("a", "b", null, "c"); + } + + @Test + public void testAbsoluteValidationEmptySubsequentPath() { + thrown.expect(IllegalArgumentException.class); + DisplayData.Path.absolute("a", "b", "", "c"); + } + + @Test + public void testPathToString() { + assertEquals("root string", "[]", DisplayData.Path.root().toString()); + assertEquals("single component", "[a]", DisplayData.Path.absolute("a").toString()); + assertEquals("hierarchy", "[a/b/c]", DisplayData.Path.absolute("a", "b", "c").toString()); + } + + @Test + public void testPathEquality() { + new EqualsTester() + .addEqualityGroup(DisplayData.Path.root(), DisplayData.Path.root()) + .addEqualityGroup(DisplayData.Path.root().extend("a"), DisplayData.Path.absolute("a")) + .addEqualityGroup( + DisplayData.Path.root().extend("a").extend("b"), + DisplayData.Path.absolute("a", "b")) + .testEquals(); + } @Test public void testIncludes() { @@ -343,69 +420,56 @@ public class DisplayDataTest implements Serializable { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.include(subComponent); + builder.include("p", subComponent); } }); - assertThat(data, includesDisplayDataFrom(subComponent)); + assertThat(data, includesDisplayDataFor("p", subComponent)); } @Test - public void testIncludesNamespaceOverride() { - final HasDisplayData subComponent = new HasDisplayData() { - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.add(DisplayData.item("foo", "bar")); - } - }; - - final HasDisplayData namespaceOverride = new HasDisplayData(){ + public void testIncludeSameComponentAtDifferentPaths() { + final HasDisplayData subComponent1 = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("foo", "bar")); } }; - - DisplayData data = DisplayData.from(new HasDisplayData() { - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.include(subComponent, namespaceOverride.getClass()); - } - }); - - assertThat(data, includesDisplayDataFrom(subComponent, namespaceOverride.getClass())); - } - - @Test - public void testNamespaceOverrideMultipleLevels() { - final HasDisplayData componentA = new HasDisplayData() { + final HasDisplayData subComponent2 = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add(DisplayData.item("foo", "bar")); + builder.add(DisplayData.item("foo2", "bar2")); } }; - final HasDisplayData componentB = new HasDisplayData() { + HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { builder - .add(DisplayData.item("foo", "bar")) - .include(componentA); + .include("p1", subComponent1) + .include("p2", subComponent2); + } }; - final HasDisplayData componentC = new HasDisplayData() { + DisplayData data = DisplayData.from(component); + assertThat(data, includesDisplayDataFor("p1", subComponent1)); + assertThat(data, includesDisplayDataFor("p2", subComponent2)); + } + + @Test + public void testIncludesComponentsAtSamePath() { + HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { builder - .add(DisplayData.item("foo", "bar")) - .include(componentB, "overrideB"); + .include("p", new NoopDisplayData()) + .include("p", new NoopDisplayData()); } }; - DisplayData data = DisplayData.from(componentC); - assertThat(data, hasDisplayItem(hasNamespace(componentC.getClass()))); - assertThat(data, hasDisplayItem(hasNamespace("overrideB"))); - assertThat(data, hasDisplayItem(hasNamespace(componentA.getClass()))); + thrown.expectCause(isA(IllegalArgumentException.class)); + DisplayData.from(component); } @Test @@ -416,7 +480,7 @@ public class DisplayDataTest implements Serializable { @Override public void populateDisplayData(Builder builder) { builder.add(DisplayData.item("foo", "bar") - .withNamespace((Class<?>) null)); + .withNamespace(null)); } }); } @@ -425,10 +489,14 @@ public class DisplayDataTest implements Serializable { 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")) + DisplayData.Identifier.of(DisplayData.Path.absolute("a"), DisplayDataTest.class, "1"), + DisplayData.Identifier.of(DisplayData.Path.absolute("a"), DisplayDataTest.class, "1")) + .addEqualityGroup( + DisplayData.Identifier.of(DisplayData.Path.absolute("b"), DisplayDataTest.class, "1")) + .addEqualityGroup( + DisplayData.Identifier.of(DisplayData.Path.absolute("a"), Object.class, "1")) + .addEqualityGroup( + DisplayData.Identifier.of(DisplayData.Path.absolute("a"), DisplayDataTest.class, "2")) .testEquals(); } @@ -466,30 +534,6 @@ public class DisplayDataTest implements Serializable { } @Test - public void testAnonymousClassNamespace() { - DisplayData data = - DisplayData.from( - new HasDisplayData() { - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.add(DisplayData.item("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( @@ -498,7 +542,7 @@ public class DisplayDataTest implements Serializable { public void populateDisplayData(DisplayData.Builder builder) { builder .add(DisplayData.item("foo", "bar")) - .include( + .include("p", new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { @@ -551,7 +595,7 @@ public class DisplayDataTest implements Serializable { }; DisplayData data = DisplayData.from(component); - assertEquals(String.format("%s:foo=bar", component.getClass().getName()), data.toString()); + assertEquals(String.format("[]%s:foo=bar", component.getClass().getName()), data.toString()); } @Test @@ -576,7 +620,7 @@ public class DisplayDataTest implements Serializable { new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.include(componentA); + builder.include("p", componentA); } }; @@ -588,13 +632,38 @@ public class DisplayDataTest implements Serializable { } @Test + public void testHandlesIncludeCyclesDifferentInstances() { + HasDisplayData component = + new DelegatingDisplayData( + new DelegatingDisplayData( + new NoopDisplayData())); + + DisplayData data = DisplayData.from(component); + assertThat(data.items(), hasSize(2)); + } + + private class DelegatingDisplayData implements HasDisplayData { + private final HasDisplayData subComponent; + public DelegatingDisplayData(HasDisplayData subComponent) { + this.subComponent = subComponent; + } + + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("subComponent", subComponent.getClass())) + .include("p", subComponent); + } + } + + @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")); + .include("p1", new EqualsEverything("foo1", "bar1")) + .include("p2", new EqualsEverything("foo2", "bar2")); } }); @@ -626,6 +695,44 @@ public class DisplayDataTest implements Serializable { } } + @Test + public void testDelegate() { + final HasDisplayData subcomponent = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("subCompKey", "foo")); + } + }; + + final HasDisplayData wrapped = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("wrappedKey", "bar")) + .include("p", subcomponent); + } + }; + + HasDisplayData wrapper = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.delegate(wrapped); + } + }; + + DisplayData data = DisplayData.from(wrapper); + assertThat(data, hasDisplayItem(allOf( + hasKey("wrappedKey"), + hasNamespace(wrapped.getClass()), + hasPath(/* root */) + ))); + assertThat(data, hasDisplayItem(allOf( + hasKey("subCompKey"), + hasNamespace(subcomponent.getClass()), + hasPath("p") + ))); + } + abstract static class IncludeSubComponent implements HasDisplayData { HasDisplayData subComponent; @@ -633,7 +740,7 @@ public class DisplayDataTest implements Serializable { public void populateDisplayData(DisplayData.Builder builder) { builder .add(DisplayData.item("id", getId())) - .include(subComponent); + .include(getId(), subComponent); } abstract String getId(); @@ -657,7 +764,7 @@ public class DisplayDataTest implements Serializable { } }); - Collection<Item<?>> items = data.items(); + Collection<Item> items = data.items(); assertThat( items, hasItem(allOf(hasKey("string"), hasType(DisplayData.Type.STRING)))); assertThat( @@ -813,7 +920,7 @@ public class DisplayDataTest implements Serializable { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .include(subComponent) + .include("p", subComponent) .add(DisplayData.item("alpha", "bravo")); } }; @@ -840,26 +947,31 @@ public class DisplayDataTest implements Serializable { new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.include(null); + builder.include("p", null); } }); } @Test - public void testIncludeNullNamespace() { - final HasDisplayData subComponent = new HasDisplayData() { + public void testIncludeNullPath() { + thrown.expectCause(isA(NullPointerException.class)); + DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { + builder.include(null, new NoopDisplayData()); } - }; + }); + } - thrown.expectCause(isA(NullPointerException.class)); + @Test + public void testIncludeEmptyPath() { + thrown.expectCause(isA(IllegalArgumentException.class)); DisplayData.from(new HasDisplayData() { - @Override - public void populateDisplayData(Builder builder) { - builder.include(subComponent, (Class<?>) null); - } - }); + @Override + public void populateDisplayData(Builder builder) { + builder.include("", new NoopDisplayData()); + } + }); } @Test @@ -964,6 +1076,44 @@ public class DisplayDataTest implements Serializable { quoted("DisplayDataTest"), "baz", "http://abc")); } + + @Test + public void testJsonSerializationAnonymousClassNamespace() throws IOException { + HasDisplayData component = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }; + DisplayData data = DisplayData.from(component); + + JsonNode json = MAPPER.readTree(MAPPER.writeValueAsBytes(data)); + String namespace = json.elements().next().get("namespace").asText(); + final Pattern anonClassRegex = Pattern.compile( + Pattern.quote(DisplayDataTest.class.getName()) + "\\$\\d+$"); + assertThat(namespace, 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 testCanSerializeItemSpecReference() { + DisplayData.ItemSpec<?> spec = DisplayData.item("clazz", DisplayDataTest.class); + SerializableUtils.ensureSerializable(new HoldsItemSpecReference(spec)); + } + + private static class HoldsItemSpecReference implements Serializable { + private final DisplayData.ItemSpec<?> spec; + public HoldsItemSpecReference(DisplayData.ItemSpec<?> spec) { + this.spec = spec; + } + } + /** * Verify that {@link DisplayData.Builder} can recover from exceptions thrown in user code. * This is not used within the Beam SDK since we want all code to produce valid DisplayData. @@ -993,14 +1143,14 @@ public class DisplayDataTest implements Serializable { .add(DisplayData.item("c", "c")); try { - builder.include(failingComponent); + builder.include("p", failingComponent); fail("Expected exception not thrown"); } catch (RuntimeException e) { // Expected } builder - .include(safeComponent) + .include("p", safeComponent) .add(DisplayData.item("d", "d")); } }); @@ -1033,7 +1183,7 @@ public class DisplayDataTest implements Serializable { HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.include(new HasDisplayData() { + builder.include("p", new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { throw cause; @@ -1046,18 +1196,6 @@ public class DisplayDataTest implements Serializable { DisplayData.from(component); } - private static class IdentityTransform<T> extends PTransform<PCollection<T>, PCollection<T>> { - @Override - public PCollection<T> apply(PCollection<T> input) { - return input.apply(ParDo.of(new DoFn<T, T>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - c.output(c.element()); - } - })); - } - } - private String quoted(Object obj) { return String.format("\"%s\"", obj); } @@ -1101,21 +1239,26 @@ public class DisplayDataTest implements Serializable { return hasItem(jsonNode); } - private static Matcher<DisplayData.Item<?>> hasUrl(Matcher<String> urlMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, String>( + private static class NoopDisplayData implements HasDisplayData { + @Override + public void populateDisplayData(Builder builder) {} + } + + 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) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getLinkUrl(); } }; } - private static <T> Matcher<DisplayData.Item<?>> hasShortValue(Matcher<T> valueStringMatcher) { - return new FeatureMatcher<DisplayData.Item<?>, T>( + private static <T> Matcher<DisplayData.Item> hasShortValue(Matcher<T> valueStringMatcher) { + return new FeatureMatcher<DisplayData.Item, T>( valueStringMatcher, "display item with short value", "short value") { @Override - protected T featureValueOf(DisplayData.Item<?> actual) { + protected T featureValueOf(DisplayData.Item actual) { @SuppressWarnings("unchecked") T shortValue = (T) actual.getShortValue(); return shortValue; http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index 9744fc6..30228fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -19,7 +19,7 @@ package org.apache.beam.sdk.transforms.windowing; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; @@ -256,7 +256,7 @@ public class WindowTest implements Serializable { DisplayData displayData = DisplayData.from(window); assertThat(displayData, hasDisplayItem("windowFn", windowFn.getClass())); - assertThat(displayData, includesDisplayDataFrom(windowFn)); + assertThat(displayData, includesDisplayDataFor("windowFn", windowFn)); assertThat(displayData, hasDisplayItem("trigger", triggerBuilder.toString())); assertThat(displayData, http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java ---------------------------------------------------------------------- diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 5914ba2..5626067 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2059,11 +2059,6 @@ public class BigQueryIO { c.sideOutput(multiPartitionsTag, KV.of(++partitionId, currResults)); } } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - } } /** http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java ---------------------------------------------------------------------- diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index c1b882a..90b9584 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -582,7 +582,7 @@ public class BigtableIO { @Override public void populateDisplayData(DisplayData.Builder builder) { - Write.this.populateDisplayData(builder); + builder.delegate(Write.this); } /////////////////////////////////////////////////////////////////////////////// http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ad03d07a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java ---------------------------------------------------------------------- diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index bfbff32..3727f92 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -802,7 +802,7 @@ public class DatastoreV1 { builder .addIfNotNull(DisplayData.item("projectId", projectId) .withLabel("Output Project")) - .include(mutationFn); + .include("mutationFn", mutationFn); } public String getProjectId() {
