[BEAM-1551] Allow `PAssert`s to take a message
Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/e3cafb42 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/e3cafb42 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/e3cafb42 Branch: refs/heads/master Commit: e3cafb42cdea1a35e067704e3a8f1277549aff67 Parents: 818fc94 Author: Aviem Zur <[email protected]> Authored: Sun Feb 26 19:42:13 2017 +0200 Committer: Aviem Zur <[email protected]> Committed: Fri Mar 10 23:13:38 2017 +0200 ---------------------------------------------------------------------- .../org/apache/beam/sdk/testing/PAssert.java | 168 ++++++++++++++----- .../apache/beam/sdk/testing/PAssertTest.java | 3 +- 2 files changed, 130 insertions(+), 41 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/e3cafb42/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index a6fb232e..1faa024 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -279,7 +279,14 @@ public class PAssert { * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}. */ public static <T> IterableAssert<T> that(PCollection<T> actual) { - return new PCollectionContentsAssert<>(actual); + return that(actual.getName(), actual); + } + + /** + * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}. + */ + public static <T> IterableAssert<T> that(String reason, PCollection<T> actual) { + return new PCollectionContentsAssert<>(reason, actual); } /** @@ -288,6 +295,15 @@ public class PAssert { */ public static <T> IterableAssert<T> thatSingletonIterable( PCollection<? extends Iterable<T>> actual) { + return thatSingletonIterable(actual.getName(), actual); + } + + /** + * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection} which + * must contain a single {@code Iterable<T>} value. + */ + public static <T> IterableAssert<T> thatSingletonIterable( + String reason, PCollection<? extends Iterable<T>> actual) { try { } catch (NoSuchElementException | IllegalArgumentException exc) { @@ -300,7 +316,7 @@ public class PAssert { @SuppressWarnings("unchecked") // Safe covariant cast PCollection<Iterable<T>> actualIterables = (PCollection<Iterable<T>>) actual; - return new PCollectionSingletonIterableAssert<>(actualIterables); + return new PCollectionSingletonIterableAssert<>(reason, actualIterables); } /** @@ -308,7 +324,15 @@ public class PAssert { * {@code PCollection PCollection<T>}, which must be a singleton. */ public static <T> SingletonAssert<T> thatSingleton(PCollection<T> actual) { - return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder()); + return thatSingleton(actual.getName(), actual); + } + + /** + * Constructs a {@link SingletonAssert} for the value of the provided + * {@code PCollection PCollection<T>}, which must be a singleton. + */ + public static <T> SingletonAssert<T> thatSingleton(String reason, PCollection<T> actual) { + return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder(), reason); } /** @@ -319,12 +343,24 @@ public class PAssert { */ public static <K, V> SingletonAssert<Map<K, Iterable<V>>> thatMultimap( PCollection<KV<K, V>> actual) { + return thatMultimap(actual.getName(), actual); + } + + /** + * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}. + * + * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any + * {@code Coder<K, V>}. + */ + public static <K, V> SingletonAssert<Map<K, Iterable<V>>> thatMultimap( + String reason, PCollection<KV<K, V>> actual) { @SuppressWarnings("unchecked") KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder(); return new PCollectionViewAssert<>( actual, View.<K, V>asMultimap(), - MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder()))); + MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())), + reason); } /** @@ -335,10 +371,23 @@ public class PAssert { * {@code Coder<K, V>}. */ public static <K, V> SingletonAssert<Map<K, V>> thatMap(PCollection<KV<K, V>> actual) { + return thatMap(actual.getName(), actual); + } + + /** + * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}, which + * must have at most one value per key. + * + * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any + * {@code Coder<K, V>}. + */ + public static <K, V> SingletonAssert<Map<K, V>> thatMap( + String reason, PCollection<KV<K, V>> actual) { @SuppressWarnings("unchecked") KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder(); return new PCollectionViewAssert<>( - actual, View.<K, V>asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder())); + actual, View.<K, V>asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()), + reason); } //////////////////////////////////////////////////////////// @@ -351,18 +400,21 @@ public class PAssert { private final PCollection<T> actual; private final AssertionWindows rewindowingStrategy; private final SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor; + private final String reason; - public PCollectionContentsAssert(PCollection<T> actual) { - this(actual, IntoGlobalWindow.<T>of(), PaneExtractors.<T>allPanes()); + public PCollectionContentsAssert(String reason, PCollection<T> actual) { + this(actual, IntoGlobalWindow.<T>of(), PaneExtractors.<T>allPanes(), reason); } public PCollectionContentsAssert( PCollection<T> actual, AssertionWindows rewindowingStrategy, - SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor) { + SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor, + String reason) { this.actual = actual; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; + this.reason = reason; } @Override @@ -397,7 +449,7 @@ public class PAssert { Coder<BoundedWindow> windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); return new PCollectionContentsAssert<>( - actual, IntoStaticWindows.<T>of(windowCoder, window), paneExtractor); + actual, IntoStaticWindows.<T>of(windowCoder, window), paneExtractor, reason); } /** @@ -418,7 +470,7 @@ public class PAssert { */ @Override public PCollectionContentsAssert<T> containsInAnyOrder(Iterable<T> expectedElements) { - return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements); + return satisfies(new AssertContainsInAnyOrderRelation<T>(reason), expectedElements); } @Override @@ -471,7 +523,7 @@ public class PAssert { // more flexible bounds. @SuppressWarnings({"rawtypes", "unchecked"}) SerializableFunction<Iterable<T>, Void> checkerFn = - (SerializableFunction) new MatcherCheckerFn<>(matcher); + (SerializableFunction) new MatcherCheckerFn<>(reason, matcher); actual.apply( "PAssert$" + (assertCount++), new GroupThenAssert<>(checkerFn, rewindowingStrategy, paneExtractor)); @@ -479,15 +531,17 @@ public class PAssert { } private static class MatcherCheckerFn<T> implements SerializableFunction<T, Void> { - private SerializableMatcher<T> matcher; + private final String reason; + private final SerializableMatcher<T> matcher; - public MatcherCheckerFn(SerializableMatcher<T> matcher) { + public MatcherCheckerFn(String reason, SerializableMatcher<T> matcher) { + this.reason = reason; this.matcher = matcher; } @Override public Void apply(T actual) { - assertThat(actual, matcher); + assertThat(reason, actual, matcher); return null; } } @@ -526,16 +580,19 @@ public class PAssert { private final AssertionWindows rewindowingStrategy; private final SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>> paneExtractor; + private final String reason; - public PCollectionSingletonIterableAssert(PCollection<Iterable<T>> actual) { - this(actual, IntoGlobalWindow.<Iterable<T>>of(), PaneExtractors.<Iterable<T>>onlyPane()); + public PCollectionSingletonIterableAssert(String reason, PCollection<Iterable<T>> actual) { + this(actual, IntoGlobalWindow.<Iterable<T>>of(), PaneExtractors.<Iterable<T>>onlyPane(), + reason); } public PCollectionSingletonIterableAssert( PCollection<Iterable<T>> actual, AssertionWindows rewindowingStrategy, SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>> - paneExtractor) { + paneExtractor, + String reason) { this.actual = actual; @SuppressWarnings("unchecked") @@ -544,6 +601,7 @@ public class PAssert { this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; + this.reason = reason; } @Override @@ -579,7 +637,7 @@ public class PAssert { Coder<BoundedWindow> windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); return new PCollectionSingletonIterableAssert<>( - actual, IntoStaticWindows.<Iterable<T>>of(windowCoder, window), paneExtractor); + actual, IntoStaticWindows.<Iterable<T>>of(windowCoder, window), paneExtractor, reason); } @Override @@ -595,7 +653,7 @@ public class PAssert { @Override public PCollectionSingletonIterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements) { - return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements); + return satisfies(new AssertContainsInAnyOrderRelation<T>(reason), expectedElements); } @Override @@ -626,12 +684,15 @@ public class PAssert { private final SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>> paneExtractor; private final Coder<ViewT> coder; + private final String reason; protected PCollectionViewAssert( PCollection<ElemT> actual, PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view, - Coder<ViewT> coder) { - this(actual, view, IntoGlobalWindow.<ElemT>of(), PaneExtractors.<ElemT>onlyPane(), coder); + Coder<ViewT> coder, + String reason) { + this(actual, view, IntoGlobalWindow.<ElemT>of(), PaneExtractors.<ElemT>onlyPane(), coder, + reason); } private PCollectionViewAssert( @@ -639,12 +700,14 @@ public class PAssert { PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view, AssertionWindows rewindowActuals, SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>> paneExtractor, - Coder<ViewT> coder) { + Coder<ViewT> coder, + String reason) { this.actual = actual; this.view = view; this.rewindowActuals = rewindowActuals; this.paneExtractor = paneExtractor; this.coder = coder; + this.reason = reason; } @Override @@ -671,17 +734,18 @@ public class PAssert { IntoStaticWindows.of( (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(), window), paneExtractor, - coder); + coder, + reason); } @Override public PCollectionViewAssert<ElemT, ViewT> isEqualTo(ViewT expectedValue) { - return satisfies(new AssertIsEqualToRelation<ViewT>(), expectedValue); + return satisfies(new AssertIsEqualToRelation<ViewT>(reason), expectedValue); } @Override public PCollectionViewAssert<ElemT, ViewT> notEqualTo(ViewT expectedValue) { - return satisfies(new AssertNotEqualToRelation<ViewT>(), expectedValue); + return satisfies(new AssertNotEqualToRelation<ViewT>(reason), expectedValue); } @Override @@ -1119,15 +1183,17 @@ public class PAssert { * value. */ private static class AssertIsEqualTo<T> implements SerializableFunction<T, Void> { - private T expected; + private final String reason; + private final T expected; - public AssertIsEqualTo(T expected) { + public AssertIsEqualTo(String reason, T expected) { + this.reason = reason; this.expected = expected; } @Override public Void apply(T actual) { - assertThat(actual, equalTo(expected)); + assertThat(reason, actual, equalTo(expected)); return null; } } @@ -1137,15 +1203,17 @@ public class PAssert { * value. */ private static class AssertNotEqualTo<T> implements SerializableFunction<T, Void> { + private String reason; private T expected; - public AssertNotEqualTo(T expected) { + public AssertNotEqualTo(String reason, T expected) { + this.reason = reason; this.expected = expected; } @Override public Void apply(T actual) { - assertThat(actual, not(equalTo(expected))); + assertThat(reason, actual, not(equalTo(expected))); return null; } } @@ -1156,25 +1224,27 @@ public class PAssert { */ private static class AssertContainsInAnyOrder<T> implements SerializableFunction<Iterable<T>, Void> { - private T[] expected; + private final String reason; + private final T[] expected; @SafeVarargs - public AssertContainsInAnyOrder(T... expected) { + public AssertContainsInAnyOrder(String reason, T... expected) { + this.reason = reason; this.expected = expected; } @SuppressWarnings("unchecked") - public AssertContainsInAnyOrder(Collection<T> expected) { - this((T[]) expected.toArray()); + public AssertContainsInAnyOrder(String reason, Collection<T> expected) { + this(reason, (T[]) expected.toArray()); } - public AssertContainsInAnyOrder(Iterable<T> expected) { - this(Lists.<T>newArrayList(expected)); + public AssertContainsInAnyOrder(String reason, Iterable<T> expected) { + this(reason, Lists.<T>newArrayList(expected)); } @Override public Void apply(Iterable<T> actual) { - assertThat(actual, containsInAnyOrder(expected)); + assertThat(reason, actual, containsInAnyOrder(expected)); return null; } } @@ -1194,9 +1264,15 @@ public class PAssert { * An {@link AssertRelation} implementing the binary predicate that two objects are equal. */ private static class AssertIsEqualToRelation<T> implements AssertRelation<T, T> { + private final String reason; + + public AssertIsEqualToRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction<T, Void> assertFor(T expected) { - return new AssertIsEqualTo<T>(expected); + return new AssertIsEqualTo<T>(reason, expected); } } @@ -1204,9 +1280,15 @@ public class PAssert { * An {@link AssertRelation} implementing the binary predicate that two objects are not equal. */ private static class AssertNotEqualToRelation<T> implements AssertRelation<T, T> { + private final String reason; + + public AssertNotEqualToRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction<T, Void> assertFor(T expected) { - return new AssertNotEqualTo<T>(expected); + return new AssertNotEqualTo<T>(reason, expected); } } @@ -1216,9 +1298,15 @@ public class PAssert { */ private static class AssertContainsInAnyOrderRelation<T> implements AssertRelation<Iterable<T>, Iterable<T>> { + private final String reason; + + public AssertContainsInAnyOrderRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction<Iterable<T>, Void> assertFor(Iterable<T> expectedElements) { - return new AssertContainsInAnyOrder<T>(expectedElements); + return new AssertContainsInAnyOrder<T>(reason, expectedElements); } } http://git-wip-us.apache.org/repos/asf/beam/blob/e3cafb42/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index 777e1af..f50adf4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -389,10 +389,11 @@ public class PAssertTest implements Serializable { @Category(RunnableOnService.class) public void testEmptyFalse() throws Exception { PCollection<Long> vals = pipeline.apply(CountingInput.upTo(5L)); - PAssert.that(vals).empty(); + PAssert.that("Vals should have been empty", vals).empty(); Throwable thrown = runExpectingAssertionFailure(pipeline); + assertThat(thrown.getMessage(), containsString("Vals should have been empty")); assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order")); }
