Repository: beam Updated Branches: refs/heads/master 6bca928b4 -> ebd3bffcd
Add `Filter#equal` Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/f2572fa5 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/f2572fa5 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/f2572fa5 Branch: refs/heads/master Commit: f2572fa5a605917039ae5cf0387ee06b24cdc3d0 Parents: 6bca928 Author: mingmxu <[email protected]> Authored: Thu Apr 20 14:47:10 2017 -0700 Committer: Thomas Groh <[email protected]> Committed: Tue Apr 25 09:58:21 2017 -0700 ---------------------------------------------------------------------- .../org/apache/beam/sdk/transforms/Filter.java | 35 +++++++++++++++++--- .../apache/beam/sdk/transforms/FilterTest.java | 13 ++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/f2572fa5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java ---------------------------------------------------------------------- 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 a564999..d0314eb 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<T> extends PTransform<PCollection<T>, PCollection<T>> { * listOfNumbers.apply(Filter.lessThan(10)); * } </pre> * - * <p>See also {@link #lessThanEq}, {@link #greaterThanEq}, + * <p>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 class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> { * listOfNumbers.apply(Filter.greaterThan(1000)); * } </pre> * - * <p>See also {@link #greaterThanEq}, {@link #lessThan}, + * <p>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 class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> { * listOfNumbers.apply(Filter.lessThanEq(10)); * } </pre> * - * <p>See also {@link #lessThan}, {@link #greaterThanEq}, + * <p>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 class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> { * listOfNumbers.apply(Filter.greaterThanEq(1000)); * } </pre> * - * <p>See also {@link #greaterThan}, {@link #lessThan}, + * <p>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 class Filter<T> extends PTransform<PCollection<T>, PCollection<T>> { }).described(String.format("x ⥠%s", value)); } + /** + * Returns a {@code PTransform} that takes an input + * {@code PCollection<T>} and returns a {@code PCollection<T>} with + * elements that equals to a given value. Elements must be {@code Comparable}. + * + * <p>Example of use: + * <pre> {@code + * PCollection<Integer> listOfNumbers = ...; + * PCollection<Integer> equalNumbers = listOfNumbers.apply(Filter.equal(1000)); + * } </pre> + * + * <p>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. + * + * <p>See also {@link #by}, which returns elements that satisfy the given predicate. + */ + public static <T extends Comparable<T>> Filter<T> equal(final T value) { + return by(new SerializableFunction<T, Boolean>() { + @Override + public Boolean apply(T input) { + return input.compareTo(value) == 0; + } + }).described(String.format("x == %s", value)); + } + /////////////////////////////////////////////////////////////////////////////// private SerializableFunction<T, Boolean> predicate; http://git-wip-us.apache.org/repos/asf/beam/blob/f2572fa5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java ---------------------------------------------------------------------- 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 85ad796..a2c5ad5 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 @@ -139,6 +139,17 @@ public class FilterTest implements Serializable { } @Test + @Category(ValidatesRunner.class) + public void testFilterEqual() { + PCollection<Integer> 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 class FilterTest implements Serializable { 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")); } }
