Repository: beam Updated Branches: refs/heads/master 7c78480cf -> d16715309
Added assertion failure tests for `PAssert#thatSingleton` Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/59fd45b8 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/59fd45b8 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/59fd45b8 Branch: refs/heads/master Commit: 59fd45b8104c7c19c65aa0df31fb9312ff82c650 Parents: 2485a4c Author: Aviem Zur <[email protected]> Authored: Wed Mar 1 08:31:57 2017 +0200 Committer: Aviem Zur <[email protected]> Committed: Fri Mar 10 23:13:38 2017 +0200 ---------------------------------------------------------------------- .../apache/beam/sdk/testing/PAssertTest.java | 36 ++++++++++++++++++++ 1 file changed, 36 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/59fd45b8/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 9bdb1b5..1603db5 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 @@ -297,6 +297,42 @@ public class PAssertTest implements Serializable { } /** + * Test that we throw an error for false assertion on singleton. + */ + @Test + @Category(RunnableOnService.class) + public void testPAssertEqualsSingletonFalse() throws Exception { + PCollection<Integer> 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<Integer> 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. */ @Test
