Skip to content

Commit

Permalink
Better assertion error messages for PAssert.thatSingleton (#31761)
Browse files Browse the repository at this point in the history
  • Loading branch information
Amar3tto authored Jul 18, 2024
1 parent a767d41 commit 24f22f2
Show file tree
Hide file tree
Showing 3 changed files with 45 additions and 3 deletions.
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test"
"https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test"
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.Pipeline.PipelineVisitor;
import org.apache.beam.sdk.PipelineRunner;
Expand Down Expand Up @@ -1610,8 +1611,18 @@ private SingletonCheckerDoFn(

@ProcessElement
public void processElement(ProcessContext c) {
ActualT actualContents = Iterables.getOnlyElement(c.element());
c.output(doChecks(site, actualContents, checkerFn));
try {
ActualT actualContents = Iterables.getOnlyElement(c.element());
c.output(doChecks(site, actualContents, checkerFn));
} catch (NoSuchElementException e) {
c.output(
SuccessOrFailure.failure(
site,
new IllegalArgumentException(
"expected singleton PCollection but was: empty PCollection", e)));
} catch (IllegalArgumentException e) {
c.output(SuccessOrFailure.failure(site, e));
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.io.GenerateSequence;
import org.apache.beam.sdk.testing.PAssert.MatcherCheckerFn;
Expand Down Expand Up @@ -386,6 +387,36 @@ public void testPAssertEqualsSingletonFalse() throws Exception {
assertThat(message, containsString("but: was <42>"));
}

@Test
@Category({ValidatesRunner.class, UsesFailureMessage.class})
public void testPAssertEqualsSingletonFailsForEmptyPCollection() throws Exception {
PCollection<Integer> pcollection = pipeline.apply(Create.empty(VarIntCoder.of()));
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 singleton PCollection"));
assertThat(message, containsString("but was: empty PCollection"));
}

@Test
@Category({ValidatesRunner.class, UsesFailureMessage.class})
public void testPAssertEqualsSingletonFailsForNonSingletonPCollection() throws Exception {
PCollection<Integer> pcollection = pipeline.apply(Create.of(44, 44));
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 one element"));
assertThat(message, containsString("but was: <44, 44>"));
}

/** Test that we throw an error for false assertion on singleton. */
@Test
@Category({ValidatesRunner.class, UsesFailureMessage.class})
Expand Down

0 comments on commit 24f22f2

Please sign in to comment.