You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2017/03/11 15:06:35 UTC

[1/6] beam git commit: Added assertion failure tests for `PAssert#thatSingleton`

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 <av...@gmail.com>
Authored: Wed Mar 1 08:31:57 2017 +0200
Committer: Aviem Zur <av...@gmail.com>
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


[5/6] beam git commit: Remove exception suppression from PAssert.SideInputCheckerDoFn

Posted by ke...@apache.org.
Remove exception suppression from PAssert.SideInputCheckerDoFn


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3669146c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3669146c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3669146c

Branch: refs/heads/master
Commit: 3669146c95e265c0fbde4444ce7d04f6b787cdac
Parents: 59fd45b
Author: Aviem Zur <av...@gmail.com>
Authored: Fri Mar 10 23:15:15 2017 +0200
Committer: Aviem Zur <av...@gmail.com>
Committed: Fri Mar 10 23:15:15 2017 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/testing/PAssert.java  | 12 ++----------
 1 file changed, 2 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/3669146c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
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 d88c4d6..2596335 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
@@ -40,7 +40,6 @@ import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.MapCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.transforms.Aggregator;
@@ -1102,15 +1101,8 @@ public class PAssert {
 
     @ProcessElement
     public void processElement(ProcessContext c) {
-      try {
-        ActualT actualContents = c.sideInput(actual);
-        doChecks(actualContents, checkerFn, success, failure);
-      } catch (Throwable t) {
-        // Suppress exception in streaming
-        if (!c.getPipelineOptions().as(StreamingOptions.class).isStreaming()) {
-          throw t;
-        }
-      }
+      ActualT actualContents = c.sideInput(actual);
+      doChecks(actualContents, checkerFn, success, failure);
     }
   }
 


[2/6] beam git commit: [BEAM-1551] Allow `PAssert`s to take a message

Posted by ke...@apache.org.
[BEAM-1551] Allow `PAssert`s to take a message


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/e3cafb42
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/e3cafb42
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/e3cafb42

Branch: refs/heads/master
Commit: e3cafb42cdea1a35e067704e3a8f1277549aff67
Parents: 818fc94
Author: Aviem Zur <av...@gmail.com>
Authored: Sun Feb 26 19:42:13 2017 +0200
Committer: Aviem Zur <av...@gmail.com>
Committed: Fri Mar 10 23:13:38 2017 +0200

----------------------------------------------------------------------
 .../org/apache/beam/sdk/testing/PAssert.java    | 168 ++++++++++++++-----
 .../apache/beam/sdk/testing/PAssertTest.java    |   3 +-
 2 files changed, 130 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e3cafb42/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
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 a6fb232e..1faa024 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
@@ -279,7 +279,14 @@ public class PAssert {
    * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}.
    */
   public static <T> IterableAssert<T> that(PCollection<T> actual) {
-    return new PCollectionContentsAssert<>(actual);
+    return that(actual.getName(), actual);
+  }
+
+  /**
+   * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}.
+   */
+  public static <T> IterableAssert<T> that(String reason, PCollection<T> actual) {
+    return new PCollectionContentsAssert<>(reason, actual);
   }
 
   /**
@@ -288,6 +295,15 @@ public class PAssert {
    */
   public static <T> IterableAssert<T> thatSingletonIterable(
       PCollection<? extends Iterable<T>> actual) {
+    return thatSingletonIterable(actual.getName(), actual);
+  }
+
+  /**
+   * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection} which
+   * must contain a single {@code Iterable<T>} value.
+   */
+  public static <T> IterableAssert<T> thatSingletonIterable(
+      String reason, PCollection<? extends Iterable<T>> actual) {
 
     try {
     } catch (NoSuchElementException | IllegalArgumentException exc) {
@@ -300,7 +316,7 @@ public class PAssert {
     @SuppressWarnings("unchecked") // Safe covariant cast
     PCollection<Iterable<T>> actualIterables = (PCollection<Iterable<T>>) actual;
 
-    return new PCollectionSingletonIterableAssert<>(actualIterables);
+    return new PCollectionSingletonIterableAssert<>(reason, actualIterables);
   }
 
   /**
@@ -308,7 +324,15 @@ public class PAssert {
    * {@code PCollection PCollection<T>}, which must be a singleton.
    */
   public static <T> SingletonAssert<T> thatSingleton(PCollection<T> actual) {
-    return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder());
+    return thatSingleton(actual.getName(), actual);
+  }
+
+  /**
+   * Constructs a {@link SingletonAssert} for the value of the provided
+   * {@code PCollection PCollection<T>}, which must be a singleton.
+   */
+  public static <T> SingletonAssert<T> thatSingleton(String reason, PCollection<T> actual) {
+    return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder(), reason);
   }
 
   /**
@@ -319,12 +343,24 @@ public class PAssert {
    */
   public static <K, V> SingletonAssert<Map<K, Iterable<V>>> thatMultimap(
       PCollection<KV<K, V>> actual) {
+    return thatMultimap(actual.getName(), actual);
+  }
+
+  /**
+   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}.
+   *
+   * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
+   * {@code Coder<K, V>}.
+   */
+  public static <K, V> SingletonAssert<Map<K, Iterable<V>>> thatMultimap(
+      String reason, PCollection<KV<K, V>> actual) {
     @SuppressWarnings("unchecked")
     KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
     return new PCollectionViewAssert<>(
         actual,
         View.<K, V>asMultimap(),
-        MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())));
+        MapCoder.of(kvCoder.getKeyCoder(), IterableCoder.of(kvCoder.getValueCoder())),
+        reason);
   }
 
   /**
@@ -335,10 +371,23 @@ public class PAssert {
    * {@code Coder<K, V>}.
    */
   public static <K, V> SingletonAssert<Map<K, V>> thatMap(PCollection<KV<K, V>> actual) {
+    return thatMap(actual.getName(), actual);
+  }
+
+  /**
+   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}, which
+   * must have at most one value per key.
+   *
+   * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
+   * {@code Coder<K, V>}.
+   */
+  public static <K, V> SingletonAssert<Map<K, V>> thatMap(
+      String reason, PCollection<KV<K, V>> actual) {
     @SuppressWarnings("unchecked")
     KvCoder<K, V> kvCoder = (KvCoder<K, V>) actual.getCoder();
     return new PCollectionViewAssert<>(
-        actual, View.<K, V>asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()));
+        actual, View.<K, V>asMap(), MapCoder.of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()),
+        reason);
   }
 
   ////////////////////////////////////////////////////////////
@@ -351,18 +400,21 @@ public class PAssert {
     private final PCollection<T> actual;
     private final AssertionWindows rewindowingStrategy;
     private final SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor;
+    private final String reason;
 
-    public PCollectionContentsAssert(PCollection<T> actual) {
-      this(actual, IntoGlobalWindow.<T>of(), PaneExtractors.<T>allPanes());
+    public PCollectionContentsAssert(String reason, PCollection<T> actual) {
+      this(actual, IntoGlobalWindow.<T>of(), PaneExtractors.<T>allPanes(), reason);
     }
 
     public PCollectionContentsAssert(
         PCollection<T> actual,
         AssertionWindows rewindowingStrategy,
-        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor) {
+        SimpleFunction<Iterable<ValueInSingleWindow<T>>, Iterable<T>> paneExtractor,
+        String reason) {
       this.actual = actual;
       this.rewindowingStrategy = rewindowingStrategy;
       this.paneExtractor = paneExtractor;
+      this.reason = reason;
     }
 
     @Override
@@ -397,7 +449,7 @@ public class PAssert {
       Coder<BoundedWindow> windowCoder =
           (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
       return new PCollectionContentsAssert<>(
-          actual, IntoStaticWindows.<T>of(windowCoder, window), paneExtractor);
+          actual, IntoStaticWindows.<T>of(windowCoder, window), paneExtractor, reason);
     }
 
     /**
@@ -418,7 +470,7 @@ public class PAssert {
      */
     @Override
     public PCollectionContentsAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
+      return satisfies(new AssertContainsInAnyOrderRelation<T>(reason), expectedElements);
     }
 
     @Override
@@ -471,7 +523,7 @@ public class PAssert {
       // more flexible bounds.
       @SuppressWarnings({"rawtypes", "unchecked"})
       SerializableFunction<Iterable<T>, Void> checkerFn =
-          (SerializableFunction) new MatcherCheckerFn<>(matcher);
+          (SerializableFunction) new MatcherCheckerFn<>(reason, matcher);
       actual.apply(
           "PAssert$" + (assertCount++),
           new GroupThenAssert<>(checkerFn, rewindowingStrategy, paneExtractor));
@@ -479,15 +531,17 @@ public class PAssert {
     }
 
     private static class MatcherCheckerFn<T> implements SerializableFunction<T, Void> {
-      private SerializableMatcher<T> matcher;
+      private final String reason;
+      private final SerializableMatcher<T> matcher;
 
-      public MatcherCheckerFn(SerializableMatcher<T> matcher) {
+      public MatcherCheckerFn(String reason, SerializableMatcher<T> matcher) {
+        this.reason = reason;
         this.matcher = matcher;
       }
 
       @Override
       public Void apply(T actual) {
-        assertThat(actual, matcher);
+        assertThat(reason, actual, matcher);
         return null;
       }
     }
@@ -526,16 +580,19 @@ public class PAssert {
     private final AssertionWindows rewindowingStrategy;
     private final SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
         paneExtractor;
+    private final String reason;
 
-    public PCollectionSingletonIterableAssert(PCollection<Iterable<T>> actual) {
-      this(actual, IntoGlobalWindow.<Iterable<T>>of(), PaneExtractors.<Iterable<T>>onlyPane());
+    public PCollectionSingletonIterableAssert(String reason, PCollection<Iterable<T>> actual) {
+      this(actual, IntoGlobalWindow.<Iterable<T>>of(), PaneExtractors.<Iterable<T>>onlyPane(),
+          reason);
     }
 
     public PCollectionSingletonIterableAssert(
         PCollection<Iterable<T>> actual,
         AssertionWindows rewindowingStrategy,
         SimpleFunction<Iterable<ValueInSingleWindow<Iterable<T>>>, Iterable<Iterable<T>>>
-            paneExtractor) {
+            paneExtractor,
+        String reason) {
       this.actual = actual;
 
       @SuppressWarnings("unchecked")
@@ -544,6 +601,7 @@ public class PAssert {
 
       this.rewindowingStrategy = rewindowingStrategy;
       this.paneExtractor = paneExtractor;
+      this.reason = reason;
     }
 
     @Override
@@ -579,7 +637,7 @@ public class PAssert {
       Coder<BoundedWindow> windowCoder =
           (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
       return new PCollectionSingletonIterableAssert<>(
-          actual, IntoStaticWindows.<Iterable<T>>of(windowCoder, window), paneExtractor);
+          actual, IntoStaticWindows.<Iterable<T>>of(windowCoder, window), paneExtractor, reason);
     }
 
     @Override
@@ -595,7 +653,7 @@ public class PAssert {
 
     @Override
     public PCollectionSingletonIterableAssert<T> containsInAnyOrder(Iterable<T> expectedElements) {
-      return satisfies(new AssertContainsInAnyOrderRelation<T>(), expectedElements);
+      return satisfies(new AssertContainsInAnyOrderRelation<T>(reason), expectedElements);
     }
 
     @Override
@@ -626,12 +684,15 @@ public class PAssert {
     private final SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>>
         paneExtractor;
     private final Coder<ViewT> coder;
+    private final String reason;
 
     protected PCollectionViewAssert(
         PCollection<ElemT> actual,
         PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
-        Coder<ViewT> coder) {
-      this(actual, view, IntoGlobalWindow.<ElemT>of(), PaneExtractors.<ElemT>onlyPane(), coder);
+        Coder<ViewT> coder,
+        String reason) {
+      this(actual, view, IntoGlobalWindow.<ElemT>of(), PaneExtractors.<ElemT>onlyPane(), coder,
+          reason);
     }
 
     private PCollectionViewAssert(
@@ -639,12 +700,14 @@ public class PAssert {
         PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
         AssertionWindows rewindowActuals,
         SimpleFunction<Iterable<ValueInSingleWindow<ElemT>>, Iterable<ElemT>> paneExtractor,
-        Coder<ViewT> coder) {
+        Coder<ViewT> coder,
+        String reason) {
       this.actual = actual;
       this.view = view;
       this.rewindowActuals = rewindowActuals;
       this.paneExtractor = paneExtractor;
       this.coder = coder;
+      this.reason = reason;
     }
 
     @Override
@@ -671,17 +734,18 @@ public class PAssert {
           IntoStaticWindows.of(
               (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(), window),
           paneExtractor,
-          coder);
+          coder,
+          reason);
     }
 
     @Override
     public PCollectionViewAssert<ElemT, ViewT> isEqualTo(ViewT expectedValue) {
-      return satisfies(new AssertIsEqualToRelation<ViewT>(), expectedValue);
+      return satisfies(new AssertIsEqualToRelation<ViewT>(reason), expectedValue);
     }
 
     @Override
     public PCollectionViewAssert<ElemT, ViewT> notEqualTo(ViewT expectedValue) {
-      return satisfies(new AssertNotEqualToRelation<ViewT>(), expectedValue);
+      return satisfies(new AssertNotEqualToRelation<ViewT>(reason), expectedValue);
     }
 
     @Override
@@ -1119,15 +1183,17 @@ public class PAssert {
    * value.
    */
   private static class AssertIsEqualTo<T> implements SerializableFunction<T, Void> {
-    private T expected;
+    private final String reason;
+    private final T expected;
 
-    public AssertIsEqualTo(T expected) {
+    public AssertIsEqualTo(String reason, T expected) {
+      this.reason = reason;
       this.expected = expected;
     }
 
     @Override
     public Void apply(T actual) {
-      assertThat(actual, equalTo(expected));
+      assertThat(reason, actual, equalTo(expected));
       return null;
     }
   }
@@ -1137,15 +1203,17 @@ public class PAssert {
    * value.
    */
   private static class AssertNotEqualTo<T> implements SerializableFunction<T, Void> {
+    private String reason;
     private T expected;
 
-    public AssertNotEqualTo(T expected) {
+    public AssertNotEqualTo(String reason, T expected) {
+      this.reason = reason;
       this.expected = expected;
     }
 
     @Override
     public Void apply(T actual) {
-      assertThat(actual, not(equalTo(expected)));
+      assertThat(reason, actual, not(equalTo(expected)));
       return null;
     }
   }
@@ -1156,25 +1224,27 @@ public class PAssert {
    */
   private static class AssertContainsInAnyOrder<T>
       implements SerializableFunction<Iterable<T>, Void> {
-    private T[] expected;
+    private final String reason;
+    private final T[] expected;
 
     @SafeVarargs
-    public AssertContainsInAnyOrder(T... expected) {
+    public AssertContainsInAnyOrder(String reason, T... expected) {
+      this.reason = reason;
       this.expected = expected;
     }
 
     @SuppressWarnings("unchecked")
-    public AssertContainsInAnyOrder(Collection<T> expected) {
-      this((T[]) expected.toArray());
+    public AssertContainsInAnyOrder(String reason, Collection<T> expected) {
+      this(reason, (T[]) expected.toArray());
     }
 
-    public AssertContainsInAnyOrder(Iterable<T> expected) {
-      this(Lists.<T>newArrayList(expected));
+    public AssertContainsInAnyOrder(String reason, Iterable<T> expected) {
+      this(reason, Lists.<T>newArrayList(expected));
     }
 
     @Override
     public Void apply(Iterable<T> actual) {
-      assertThat(actual, containsInAnyOrder(expected));
+      assertThat(reason, actual, containsInAnyOrder(expected));
       return null;
     }
   }
@@ -1194,9 +1264,15 @@ public class PAssert {
    * An {@link AssertRelation} implementing the binary predicate that two objects are equal.
    */
   private static class AssertIsEqualToRelation<T> implements AssertRelation<T, T> {
+    private final String reason;
+
+    public AssertIsEqualToRelation(String reason) {
+      this.reason = reason;
+    }
+
     @Override
     public SerializableFunction<T, Void> assertFor(T expected) {
-      return new AssertIsEqualTo<T>(expected);
+      return new AssertIsEqualTo<T>(reason, expected);
     }
   }
 
@@ -1204,9 +1280,15 @@ public class PAssert {
    * An {@link AssertRelation} implementing the binary predicate that two objects are not equal.
    */
   private static class AssertNotEqualToRelation<T> implements AssertRelation<T, T> {
+    private final String reason;
+
+    public AssertNotEqualToRelation(String reason) {
+      this.reason = reason;
+    }
+
     @Override
     public SerializableFunction<T, Void> assertFor(T expected) {
-      return new AssertNotEqualTo<T>(expected);
+      return new AssertNotEqualTo<T>(reason, expected);
     }
   }
 
@@ -1216,9 +1298,15 @@ public class PAssert {
    */
   private static class AssertContainsInAnyOrderRelation<T>
       implements AssertRelation<Iterable<T>, Iterable<T>> {
+    private final String reason;
+
+    public AssertContainsInAnyOrderRelation(String reason) {
+      this.reason = reason;
+    }
+
     @Override
     public SerializableFunction<Iterable<T>, Void> assertFor(Iterable<T> expectedElements) {
-      return new AssertContainsInAnyOrder<T>(expectedElements);
+      return new AssertContainsInAnyOrder<T>(reason, expectedElements);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/e3cafb42/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 777e1af..f50adf4 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
@@ -389,10 +389,11 @@ public class PAssertTest implements Serializable {
   @Category(RunnableOnService.class)
   public void testEmptyFalse() throws Exception {
     PCollection<Long> vals = pipeline.apply(CountingInput.upTo(5L));
-    PAssert.that(vals).empty();
+    PAssert.that("Vals should have been empty", vals).empty();
 
     Throwable thrown = runExpectingAssertionFailure(pipeline);
 
+    assertThat(thrown.getMessage(), containsString("Vals should have been empty"));
     assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order"));
   }
 


[6/6] beam git commit: This closes #2109: Allow `PAssert`s to take a message

Posted by ke...@apache.org.
This closes #2109: Allow `PAssert`s to take a message

  Remove exception suppression from PAssert.SideInputCheckerDoFn
  Added assertion failure tests for `PAssert#thatSingleton`
  Added a test of default PAssert failure reason
  Javadoc changes
  [BEAM-1551] Allow `PAssert`s to take a message


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/d1671530
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/d1671530
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/d1671530

Branch: refs/heads/master
Commit: d1671530951bd67956f6d022b523053c12a04df3
Parents: 7c78480 3669146
Author: Kenneth Knowles <kl...@google.com>
Authored: Sat Mar 11 07:04:30 2017 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Sat Mar 11 07:04:30 2017 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/testing/PAssert.java    | 183 ++++++++++++++-----
 .../apache/beam/sdk/testing/PAssertTest.java    |  56 +++++-
 2 files changed, 188 insertions(+), 51 deletions(-)
----------------------------------------------------------------------



[3/6] beam git commit: Added a test of default PAssert failure reason

Posted by ke...@apache.org.
Added a test of default PAssert failure reason


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/2485a4cc
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/2485a4cc
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/2485a4cc

Branch: refs/heads/master
Commit: 2485a4ccce18784abf520792ea36b7d3d6efc394
Parents: 0d08d2a
Author: Aviem Zur <av...@gmail.com>
Authored: Wed Mar 1 08:10:03 2017 +0200
Committer: Aviem Zur <av...@gmail.com>
Committed: Fri Mar 10 23:13:38 2017 +0200

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/PAssertTest.java    | 21 ++++++++++++++++++--
 1 file changed, 19 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/2485a4cc/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 f50adf4..9bdb1b5 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
@@ -393,8 +393,25 @@ public class PAssertTest implements Serializable {
 
     Throwable thrown = runExpectingAssertionFailure(pipeline);
 
-    assertThat(thrown.getMessage(), containsString("Vals should have been empty"));
-    assertThat(thrown.getMessage(), containsString("Expected: iterable over [] in any order"));
+    String message = thrown.getMessage();
+
+    assertThat(message, containsString("Vals should have been empty"));
+    assertThat(message, containsString("Expected: iterable over [] in any order"));
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testEmptyFalseDefaultReasonString() throws Exception {
+    PCollection<Long> vals = pipeline.apply(CountingInput.upTo(5L));
+    PAssert.that(vals).empty();
+
+    Throwable thrown = runExpectingAssertionFailure(pipeline);
+
+    String message = thrown.getMessage();
+
+    assertThat(message,
+        containsString("CountingInput.BoundedCountingInput/Read(BoundedCountingSource).out"));
+    assertThat(message, containsString("Expected: iterable over [] in any order"));
   }
 
   private static Throwable runExpectingAssertionFailure(Pipeline pipeline) {


[4/6] beam git commit: Javadoc changes

Posted by ke...@apache.org.
Javadoc changes


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/0d08d2a4
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/0d08d2a4
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/0d08d2a4

Branch: refs/heads/master
Commit: 0d08d2a4561b286711ea4322f8d53ce5ad11e89f
Parents: e3cafb4
Author: Aviem Zur <av...@gmail.com>
Authored: Wed Mar 1 07:51:05 2017 +0200
Committer: Aviem Zur <av...@gmail.com>
Committed: Fri Mar 10 23:13:38 2017 +0200

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/testing/PAssert.java  | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/0d08d2a4/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
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 1faa024..d88c4d6 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
@@ -283,7 +283,8 @@ public class PAssert {
   }
 
   /**
-   * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}.
+   * Constructs an {@link IterableAssert} for the elements of the provided {@link PCollection}
+   * with the specified reason.
    */
   public static <T> IterableAssert<T> that(String reason, PCollection<T> actual) {
     return new PCollectionContentsAssert<>(reason, actual);
@@ -299,8 +300,8 @@ public class PAssert {
   }
 
   /**
-   * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection} which
-   * must contain a single {@code Iterable<T>} value.
+   * Constructs an {@link IterableAssert} for the value of the provided {@link PCollection } with
+   * the specified reason. The provided PCollection must contain a single {@code Iterable<T>} value.
    */
   public static <T> IterableAssert<T> thatSingletonIterable(
       String reason, PCollection<? extends Iterable<T>> actual) {
@@ -329,7 +330,8 @@ public class PAssert {
 
   /**
    * Constructs a {@link SingletonAssert} for the value of the provided
-   * {@code PCollection PCollection<T>}, which must be a singleton.
+   * {@code PCollection PCollection<T>} with the specified reason. The provided PCollection must be
+   * a singleton.
    */
   public static <T> SingletonAssert<T> thatSingleton(String reason, PCollection<T> actual) {
     return new PCollectionViewAssert<>(actual, View.<T>asSingleton(), actual.getCoder(), reason);
@@ -347,7 +349,8 @@ public class PAssert {
   }
 
   /**
-   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}.
+   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection} with the
+   * specified reason.
    *
    * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
    * {@code Coder<K, V>}.
@@ -375,8 +378,8 @@ public class PAssert {
   }
 
   /**
-   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection}, which
-   * must have at most one value per key.
+   * Constructs a {@link SingletonAssert} for the value of the provided {@link PCollection} with
+   * the specified reason. The {@link PCollection} must have at most one value per key.
    *
    * <p>Note that the actual value must be coded by a {@link KvCoder}, not just any
    * {@code Coder<K, V>}.