From e3cafb42cdea1a35e067704e3a8f1277549aff67 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 26 Feb 2017 19:42:13 +0200 Subject: [PATCH 1/5] [BEAM-1551] Allow `PAssert`s to take a message --- .../org/apache/beam/sdk/testing/PAssert.java | 168 +++++++++++++----- .../apache/beam/sdk/testing/PAssertTest.java | 3 +- 2 files changed, 130 insertions(+), 41 deletions(-) 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 a6fb232e6b3e..1faa0240047b 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 interface SingletonAssert { * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}. */ public static IterableAssert that(PCollection actual) { - return new PCollectionContentsAssert<>(actual); + return that(actual.getName(), actual); + } + + /** + * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}. + */ + public static IterableAssert that(String reason, PCollection actual) { + return new PCollectionContentsAssert<>(reason, actual); } /** @@ -288,6 +295,15 @@ public static IterableAssert that(PCollection actual) { */ public static IterableAssert thatSingletonIterable( PCollection> 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} value. + */ + public static IterableAssert thatSingletonIterable( + String reason, PCollection> actual) { try { } catch (NoSuchElementException | IllegalArgumentException exc) { @@ -300,7 +316,7 @@ public static IterableAssert thatSingletonIterable( @SuppressWarnings("unchecked") // Safe covariant cast PCollection> actualIterables = (PCollection>) actual; - return new PCollectionSingletonIterableAssert<>(actualIterables); + return new PCollectionSingletonIterableAssert<>(reason, actualIterables); } /** @@ -308,7 +324,15 @@ public static IterableAssert thatSingletonIterable( * {@code PCollection PCollection}, which must be a singleton. */ public static SingletonAssert thatSingleton(PCollection actual) { - return new PCollectionViewAssert<>(actual, View.asSingleton(), actual.getCoder()); + return thatSingleton(actual.getName(), actual); + } + + /** + * Constructs a {@link SingletonAssert} for the value of the provided + * {@code PCollection PCollection}, which must be a singleton. + */ + public static SingletonAssert thatSingleton(String reason, PCollection actual) { + return new PCollectionViewAssert<>(actual, View.asSingleton(), actual.getCoder(), reason); } /** @@ -319,12 +343,24 @@ public static SingletonAssert thatSingleton(PCollection actual) { */ public static SingletonAssert>> thatMultimap( PCollection> actual) { + return thatMultimap(actual.getName(), actual); + } + + /** + * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}. + * + *

Note that the actual value must be coded by a {@link KvCoder}, not just any + * {@code Coder}. + */ + public static SingletonAssert>> thatMultimap( + String reason, PCollection> actual) { @SuppressWarnings("unchecked") KvCoder kvCoder = (KvCoder) actual.getCoder(); return new PCollectionViewAssert<>( actual, View.asMultimap(), - MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder()))); + MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())), + reason); } /** @@ -335,10 +371,23 @@ public static SingletonAssert>> thatMultimap( * {@code Coder}. */ public static SingletonAssert> thatMap(PCollection> 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. + * + *

Note that the actual value must be coded by a {@link KvCoder}, not just any + * {@code Coder}. + */ + public static SingletonAssert> thatMap( + String reason, PCollection> actual) { @SuppressWarnings("unchecked") KvCoder kvCoder = (KvCoder) actual.getCoder(); return new PCollectionViewAssert<>( - actual, View.asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder())); + actual, View.asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()), + reason); } //////////////////////////////////////////////////////////// @@ -351,18 +400,21 @@ private static class PCollectionContentsAssert implements IterableAssert { private final PCollection actual; private final AssertionWindows rewindowingStrategy; private final SimpleFunction>, Iterable> paneExtractor; + private final String reason; - public PCollectionContentsAssert(PCollection actual) { - this(actual, IntoGlobalWindow.of(), PaneExtractors.allPanes()); + public PCollectionContentsAssert(String reason, PCollection actual) { + this(actual, IntoGlobalWindow.of(), PaneExtractors.allPanes(), reason); } public PCollectionContentsAssert( PCollection actual, AssertionWindows rewindowingStrategy, - SimpleFunction>, Iterable> paneExtractor) { + SimpleFunction>, Iterable> paneExtractor, + String reason) { this.actual = actual; this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; + this.reason = reason; } @Override @@ -397,7 +449,7 @@ private PCollectionContentsAssert withPane( Coder windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); return new PCollectionContentsAssert<>( - actual, IntoStaticWindows.of(windowCoder, window), paneExtractor); + actual, IntoStaticWindows.of(windowCoder, window), paneExtractor, reason); } /** @@ -418,7 +470,7 @@ public final PCollectionContentsAssert containsInAnyOrder(T... expectedElemen */ @Override public PCollectionContentsAssert containsInAnyOrder(Iterable expectedElements) { - return satisfies(new AssertContainsInAnyOrderRelation(), expectedElements); + return satisfies(new AssertContainsInAnyOrderRelation(reason), expectedElements); } @Override @@ -471,7 +523,7 @@ PCollectionContentsAssert satisfies( // more flexible bounds. @SuppressWarnings({"rawtypes", "unchecked"}) SerializableFunction, Void> checkerFn = - (SerializableFunction) new MatcherCheckerFn<>(matcher); + (SerializableFunction) new MatcherCheckerFn<>(reason, matcher); actual.apply( "PAssert$" + (assertCount++), new GroupThenAssert<>(checkerFn, rewindowingStrategy, paneExtractor)); @@ -479,15 +531,17 @@ PCollectionContentsAssert satisfies( } private static class MatcherCheckerFn implements SerializableFunction { - private SerializableMatcher matcher; + private final String reason; + private final SerializableMatcher matcher; - public MatcherCheckerFn(SerializableMatcher matcher) { + public MatcherCheckerFn(String reason, SerializableMatcher 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 @@ private static class PCollectionSingletonIterableAssert implements IterableAs private final AssertionWindows rewindowingStrategy; private final SimpleFunction>>, Iterable>> paneExtractor; + private final String reason; - public PCollectionSingletonIterableAssert(PCollection> actual) { - this(actual, IntoGlobalWindow.>of(), PaneExtractors.>onlyPane()); + public PCollectionSingletonIterableAssert(String reason, PCollection> actual) { + this(actual, IntoGlobalWindow.>of(), PaneExtractors.>onlyPane(), + reason); } public PCollectionSingletonIterableAssert( PCollection> actual, AssertionWindows rewindowingStrategy, SimpleFunction>>, Iterable>> - paneExtractor) { + paneExtractor, + String reason) { this.actual = actual; @SuppressWarnings("unchecked") @@ -544,6 +601,7 @@ public PCollectionSingletonIterableAssert( this.rewindowingStrategy = rewindowingStrategy; this.paneExtractor = paneExtractor; + this.reason = reason; } @Override @@ -579,7 +637,7 @@ private PCollectionSingletonIterableAssert withPanes( Coder windowCoder = (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(); return new PCollectionSingletonIterableAssert<>( - actual, IntoStaticWindows.>of(windowCoder, window), paneExtractor); + actual, IntoStaticWindows.>of(windowCoder, window), paneExtractor, reason); } @Override @@ -595,7 +653,7 @@ public PCollectionSingletonIterableAssert empty() { @Override public PCollectionSingletonIterableAssert containsInAnyOrder(Iterable expectedElements) { - return satisfies(new AssertContainsInAnyOrderRelation(), expectedElements); + return satisfies(new AssertContainsInAnyOrderRelation(reason), expectedElements); } @Override @@ -626,12 +684,15 @@ private static class PCollectionViewAssert implements SingletonAss private final SimpleFunction>, Iterable> paneExtractor; private final Coder coder; + private final String reason; protected PCollectionViewAssert( PCollection actual, PTransform, PCollectionView> view, - Coder coder) { - this(actual, view, IntoGlobalWindow.of(), PaneExtractors.onlyPane(), coder); + Coder coder, + String reason) { + this(actual, view, IntoGlobalWindow.of(), PaneExtractors.onlyPane(), coder, + reason); } private PCollectionViewAssert( @@ -639,12 +700,14 @@ private PCollectionViewAssert( PTransform, PCollectionView> view, AssertionWindows rewindowActuals, SimpleFunction>, Iterable> paneExtractor, - Coder coder) { + Coder 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 @@ private PCollectionViewAssert inPane( IntoStaticWindows.of( (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(), window), paneExtractor, - coder); + coder, + reason); } @Override public PCollectionViewAssert isEqualTo(ViewT expectedValue) { - return satisfies(new AssertIsEqualToRelation(), expectedValue); + return satisfies(new AssertIsEqualToRelation(reason), expectedValue); } @Override public PCollectionViewAssert notEqualTo(ViewT expectedValue) { - return satisfies(new AssertNotEqualToRelation(), expectedValue); + return satisfies(new AssertNotEqualToRelation(reason), expectedValue); } @Override @@ -1119,15 +1183,17 @@ private static void doChecks( * value. */ private static class AssertIsEqualTo implements SerializableFunction { - 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 Void apply(T actual) { * value. */ private static class AssertNotEqualTo implements SerializableFunction { + 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 Void apply(T actual) { */ private static class AssertContainsInAnyOrder implements SerializableFunction, 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 expected) { - this((T[]) expected.toArray()); + public AssertContainsInAnyOrder(String reason, Collection expected) { + this(reason, (T[]) expected.toArray()); } - public AssertContainsInAnyOrder(Iterable expected) { - this(Lists.newArrayList(expected)); + public AssertContainsInAnyOrder(String reason, Iterable expected) { + this(reason, Lists.newArrayList(expected)); } @Override public Void apply(Iterable actual) { - assertThat(actual, containsInAnyOrder(expected)); + assertThat(reason, actual, containsInAnyOrder(expected)); return null; } } @@ -1194,9 +1264,15 @@ private interface AssertRelation extends Serializable { * An {@link AssertRelation} implementing the binary predicate that two objects are equal. */ private static class AssertIsEqualToRelation implements AssertRelation { + private final String reason; + + public AssertIsEqualToRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction assertFor(T expected) { - return new AssertIsEqualTo(expected); + return new AssertIsEqualTo(reason, expected); } } @@ -1204,9 +1280,15 @@ public SerializableFunction assertFor(T expected) { * An {@link AssertRelation} implementing the binary predicate that two objects are not equal. */ private static class AssertNotEqualToRelation implements AssertRelation { + private final String reason; + + public AssertNotEqualToRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction assertFor(T expected) { - return new AssertNotEqualTo(expected); + return new AssertNotEqualTo(reason, expected); } } @@ -1216,9 +1298,15 @@ public SerializableFunction assertFor(T expected) { */ private static class AssertContainsInAnyOrderRelation implements AssertRelation, Iterable> { + private final String reason; + + public AssertContainsInAnyOrderRelation(String reason) { + this.reason = reason; + } + @Override public SerializableFunction, Void> assertFor(Iterable expectedElements) { - return new AssertContainsInAnyOrder(expectedElements); + return new AssertContainsInAnyOrder(reason, expectedElements); } } 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 777e1af8a13d..f50adf46be42 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 void testContainsInAnyOrderFalse() throws Exception { @Category(RunnableOnService.class) public void testEmptyFalse() throws Exception { PCollection 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")); } From 0d08d2a4561b286711ea4322f8d53ce5ad11e89f Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Wed, 1 Mar 2017 07:51:05 +0200 Subject: [PATCH 2/5] Javadoc changes --- .../org/apache/beam/sdk/testing/PAssert.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) 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 1faa0240047b..d88c4d658958 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 @@ -283,7 +283,8 @@ public static IterableAssert that(PCollection actual) { } /** - * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}. + * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection} + * with the specified reason. */ public static IterableAssert that(String reason, PCollection actual) { return new PCollectionContentsAssert<>(reason, actual); @@ -299,8 +300,8 @@ public static IterableAssert thatSingletonIterable( } /** - * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection} which - * must contain a single {@code Iterable} value. + * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection } with + * the specified reason. The provided PCollection must contain a single {@code Iterable} value. */ public static IterableAssert thatSingletonIterable( String reason, PCollection> actual) { @@ -329,7 +330,8 @@ public static SingletonAssert thatSingleton(PCollection actual) { /** * Constructs a {@link SingletonAssert} for the value of the provided - * {@code PCollection PCollection}, which must be a singleton. + * {@code PCollection PCollection} with the specified reason. The provided PCollection must be + * a singleton. */ public static SingletonAssert thatSingleton(String reason, PCollection actual) { return new PCollectionViewAssert<>(actual, View.asSingleton(), actual.getCoder(), reason); @@ -347,7 +349,8 @@ public static SingletonAssert>> thatMultimap( } /** - * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}. + * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection} with the + * specified reason. * *

Note that the actual value must be coded by a {@link KvCoder}, not just any * {@code Coder}. @@ -375,8 +378,8 @@ public static SingletonAssert> thatMap(PCollection> ac } /** - * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}, which - * must have at most one value per key. + * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection} with + * the specified reason. The {@link PCollection} must have at most one value per key. * *

Note that the actual value must be coded by a {@link KvCoder}, not just any * {@code Coder}. From 2485a4ccce18784abf520792ea36b7d3d6efc394 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Wed, 1 Mar 2017 08:10:03 +0200 Subject: [PATCH 3/5] Added a test of default PAssert failure reason --- .../apache/beam/sdk/testing/PAssertTest.java | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) 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 f50adf46be42..9bdb1b5921e5 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 @@ -393,8 +393,25 @@ public void testEmptyFalse() throws Exception { Throwable thrown = runExpectingAssertionFailure(pipeline); - assertThat(thrown.getMessage(), containsString("Vals should have been empty")); - assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order")); + String message = thrown.getMessage(); + + assertThat(message, containsString("Vals should have been empty")); + assertThat(message, containsString("Expected: iterable over [] in any order")); + } + + @Test + @Category(RunnableOnService.class) + public void testEmptyFalseDefaultReasonString() throws Exception { + PCollection vals = pipeline.apply(CountingInput.upTo(5L)); + PAssert.that(vals).empty(); + + Throwable thrown = runExpectingAssertionFailure(pipeline); + + String message = thrown.getMessage(); + + assertThat(message, + containsString("CountingInput.BoundedCountingInput/Read(BoundedCountingSource).out")); + assertThat(message, containsString("Expected: iterable over [] in any order")); } private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { From 59fd45b8104c7c19c65aa0df31fb9312ff82c650 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Wed, 1 Mar 2017 08:31:57 +0200 Subject: [PATCH 4/5] Added assertion failure tests for `PAssert#thatSingleton` --- .../apache/beam/sdk/testing/PAssertTest.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) 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 9bdb1b5921e5..1603db55b1d0 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 @@ -296,6 +296,42 @@ public void testNotEqualTo() throws Exception { pipeline.run(); } + /** + * Test that we throw an error for false assertion on singleton. + */ + @Test + @Category(RunnableOnService.class) + public void testPAssertEqualsSingletonFalse() throws Exception { + PCollection pcollection = pipeline.apply(Create.of(42)); + PAssert.thatSingleton("The value was not equal to 44", pcollection).isEqualTo(44); + + Throwable thrown = runExpectingAssertionFailure(pipeline); + + String message = thrown.getMessage(); + + assertThat(message, containsString("The value was not equal to 44")); + assertThat(message, containsString("Expected: <44>")); + assertThat(message, containsString("but: was <42>")); + } + + /** + * Test that we throw an error for false assertion on singleton. + */ + @Test + @Category(RunnableOnService.class) + public void testPAssertEqualsSingletonFalseDefaultReasonString() throws Exception { + PCollection pcollection = pipeline.apply(Create.of(42)); + PAssert.thatSingleton(pcollection).isEqualTo(44); + + Throwable thrown = runExpectingAssertionFailure(pipeline); + + String message = thrown.getMessage(); + + assertThat(message, containsString("Create.Values/Read(CreateSource).out")); + assertThat(message, containsString("Expected: <44>")); + assertThat(message, containsString("but: was <42>")); + } + /** * Tests that {@code containsInAnyOrder} is actually order-independent. */ From 3669146c95e265c0fbde4444ce7d04f6b787cdac Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Fri, 10 Mar 2017 23:15:15 +0200 Subject: [PATCH 5/5] Remove exception suppression from PAssert.SideInputCheckerDoFn --- .../java/org/apache/beam/sdk/testing/PAssert.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) 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 d88c4d658958..259633554b77 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 @@ -40,7 +40,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Aggregator; @@ -1102,15 +1101,8 @@ private SideInputCheckerDoFn( @ProcessElement public void processElement(ProcessContext c) { - try { - ActualT actualContents = c.sideInput(actual); - doChecks(actualContents, checkerFn, success, failure); - } catch (Throwable t) { - // Suppress exception in streaming - if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) { - throw t; - } - } + ActualT actualContents = c.sideInput(actual); + doChecks(actualContents, checkerFn, success, failure); } }