Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Better assertion error messages for PAssert.thatSingleton #31761

Merged
merged 1 commit into from
Jul 18, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
Loading