From 6fd1d57b187915d92e07e051da0550c63fb2be66 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Thu, 20 Apr 2017 14:47:10 -0700 Subject: [PATCH] Add `equal` method to Filter transform to trigger travis-ci --- .../apache/beam/sdk/transforms/Filter.java | 35 ++++++++++++++++--- .../beam/sdk/transforms/FilterTest.java | 13 +++++++ 2 files changed, 44 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java index a564999f8ec8..d0314eb8ce09 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java @@ -67,7 +67,7 @@ public class Filter extends PTransform, PCollection> { * listOfNumbers.apply(Filter.lessThan(10)); * } * - *

See also {@link #lessThanEq}, {@link #greaterThanEq}, + *

See also {@link #lessThanEq}, {@link #greaterThanEq}, {@link #equal} * and {@link #greaterThan}, which return elements satisfying various * inequalities with the specified value based on the elements' * natural ordering. @@ -98,7 +98,7 @@ public Boolean apply(T input) { * listOfNumbers.apply(Filter.greaterThan(1000)); * } * - *

See also {@link #greaterThanEq}, {@link #lessThan}, + *

See also {@link #greaterThanEq}, {@link #lessThan}, {@link #equal} * and {@link #lessThanEq}, which return elements satisfying various * inequalities with the specified value based on the elements' * natural ordering. @@ -128,7 +128,7 @@ public Boolean apply(T input) { * listOfNumbers.apply(Filter.lessThanEq(10)); * } * - *

See also {@link #lessThan}, {@link #greaterThanEq}, + *

See also {@link #lessThan}, {@link #greaterThanEq}, {@link #equal} * and {@link #greaterThan}, which return elements satisfying various * inequalities with the specified value based on the elements' * natural ordering. @@ -158,7 +158,7 @@ public Boolean apply(T input) { * listOfNumbers.apply(Filter.greaterThanEq(1000)); * } * - *

See also {@link #greaterThan}, {@link #lessThan}, + *

See also {@link #greaterThan}, {@link #lessThan}, {@link #equal} * and {@link #lessThanEq}, which return elements satisfying various * inequalities with the specified value based on the elements' * natural ordering. @@ -175,6 +175,33 @@ public Boolean apply(T input) { }).described(String.format("x ≥ %s", value)); } + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection} and returns a {@code PCollection} with + * elements that equals to a given value. Elements must be {@code Comparable}. + * + *

Example of use: + *

 {@code
+   * PCollection listOfNumbers = ...;
+   * PCollection equalNumbers = listOfNumbers.apply(Filter.equal(1000));
+   * } 
+ * + *

See also {@link #greaterThan}, {@link #lessThan}, {@link #lessThanEq} + * and {@link #greaterThanEq}, which return elements satisfying various + * inequalities with the specified value based on the elements' + * natural ordering. + * + *

See also {@link #by}, which returns elements that satisfy the given predicate. + */ + public static > Filter equal(final T value) { + return by(new SerializableFunction() { + @Override + public Boolean apply(T input) { + return input.compareTo(value) == 0; + } + }).described(String.format("x == %s", value)); + } + /////////////////////////////////////////////////////////////////////////////// private SerializableFunction predicate; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java index 85ad79678aa9..a2c5ad532609 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java @@ -138,6 +138,17 @@ public void testFilterGreaterThanEq() { p.run(); } + @Test + @Category(ValidatesRunner.class) + public void testFilterEqual() { + PCollection output = p + .apply(Create.of(1, 2, 3, 4, 5, 6, 7)) + .apply(Filter.equal(4)); + + PAssert.that(output).containsInAnyOrder(4); + p.run(); + } + @Test public void testDisplayData() { assertThat(DisplayData.from(Filter.lessThan(123)), hasDisplayItem("predicate", "x < 123")); @@ -147,5 +158,7 @@ public void testDisplayData() { assertThat(DisplayData.from(Filter.greaterThan(345)), hasDisplayItem("predicate", "x > 345")); assertThat(DisplayData.from(Filter.greaterThanEq(456)), hasDisplayItem("predicate", "x ≥ 456")); + + assertThat(DisplayData.from(Filter.equal(567)), hasDisplayItem("predicate", "x == 567")); } }