This is an automated email from the ASF dual-hosted git repository.

yhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 24f22f2b2cf Better assertion error messages for PAssert.thatSingleton 
(#31761)
24f22f2b2cf is described below

commit 24f22f2b2cf999d3533889d66b26f9f50c776c5d
Author: Vitaly Terentyev <vitaly.terent...@akvelon.com>
AuthorDate: Thu Jul 18 19:18:18 2024 +0400

    Better assertion error messages for PAssert.thatSingleton (#31761)
---
 ...eam_PostCommit_Java_ValidatesRunner_Direct.json |  2 +-
 .../java/org/apache/beam/sdk/testing/PAssert.java  | 15 +++++++++--
 .../org/apache/beam/sdk/testing/PAssertTest.java   | 31 ++++++++++++++++++++++
 3 files changed, 45 insertions(+), 3 deletions(-)

diff --git 
a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json 
b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json
index b970762c839..38ae94aee2f 100644
--- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json
+++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json
@@ -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"
 }
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 aa7b2630cce..7a102747b9f 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
@@ -31,6 +31,7 @@ import java.util.Collection;
 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;
@@ -1610,8 +1611,18 @@ public class PAssert {
 
     @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));
+      }
     }
   }
 
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 dfdb6282b54..a02196bb2c0 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
@@ -37,6 +37,7 @@ 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.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;
@@ -386,6 +387,36 @@ public class PAssertTest implements Serializable {
     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})

Reply via email to