Add Tests for PAssert.IterableAssert#empty() These tests demonstrate that an empty PCollection has no elements, and that the empty() assertion affirmatively fails when there are elements within the PCollection.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e0764767 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e0764767 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e0764767 Branch: refs/heads/apex-runner Commit: e0764767e1806b3ac82064ecc108c3944bfe8a22 Parents: b93ac70 Author: Thomas Groh <tg...@google.com> Authored: Wed Nov 2 10:35:27 2016 -0700 Committer: Thomas Groh <tg...@google.com> Committed: Thu Nov 3 15:33:43 2016 -0700 ---------------------------------------------------------------------- .../apache/beam/sdk/testing/PAssertTest.java | 27 ++++++++++++++++++++ 1 file changed, 27 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e0764767/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 54ddd3f..be8924f 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.testing; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -32,6 +33,8 @@ import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -355,6 +358,18 @@ public class PAssertTest implements Serializable { pipeline.run(); } + @Test + @Category(RunnableOnService.class) + public void testEmpty() { + Pipeline p = TestPipeline.create(); + PCollection<Long> vals = + p.apply(Create.<Long>of().withCoder(VarLongCoder.of())); + + PAssert.that(vals).empty(); + + p.run(); + } + /** * Tests that {@code containsInAnyOrder} fails when and how it should. */ @@ -381,6 +396,18 @@ public class PAssertTest implements Serializable { expectedPattern.matcher(exc.getMessage()).find()); } + @Test + @Category(RunnableOnService.class) + public void testEmptyFalse() throws Exception { + Pipeline p = TestPipeline.create(); + PCollection<Long> vals = p.apply(CountingInput.upTo(5L)); + PAssert.that(vals).empty(); + + Throwable thrown = runExpectingAssertionFailure(p); + + assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order")); + } + private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { // We cannot use thrown.expect(AssertionError.class) because the AssertionError // is first caught by JUnit and causes a test failure.