This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new ea3d2734a PARQUET-34: implement not() for Contains predicate (#2941)
ea3d2734a is described below
commit ea3d2734aaa1f8addaeced8e152a09602f2aafa7
Author: Claire McGinty <[email protected]>
AuthorDate: Mon Jul 29 22:05:02 2024 -0400
PARQUET-34: implement not() for Contains predicate (#2941)
---
.../filter2/predicate/ContainsRewriter.java | 34 ++--
.../parquet/filter2/predicate/LogicalInverter.java | 2 +-
.../parquet/filter2/predicate/Operators.java | 58 +++++-
.../IncrementallyUpdatedFilterPredicate.java | 78 +++++++++
.../column/columnindex/ColumnIndexBuilder.java | 6 +-
.../filter2/columnindex/ColumnIndexFilter.java | 2 +-
.../filter2/predicate/TestFilterApiMethods.java | 29 ++-
...crementallyUpdatedFilterPredicateGenerator.java | 195 +++++++++++----------
.../filter2/bloomfilterlevel/BloomFilterImpl.java | 2 +-
.../filter2/dictionarylevel/DictionaryFilter.java | 2 +-
.../filter2/statisticslevel/StatisticsFilter.java | 2 +-
.../recordlevel/TestRecordLevelFilters.java | 109 +++++++++++-
12 files changed, 384 insertions(+), 135 deletions(-)
diff --git
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/ContainsRewriter.java
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/ContainsRewriter.java
index ea2d70e8e..5050ee8f0 100644
---
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/ContainsRewriter.java
+++
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/ContainsRewriter.java
@@ -107,7 +107,7 @@ public final class ContainsRewriter implements
Visitor<FilterPredicate> {
} else if (and.getLeft() instanceof Contains) {
left = and.getLeft();
} else {
- return and;
+ left = and.getLeft();
}
final FilterPredicate right;
@@ -118,15 +118,18 @@ public final class ContainsRewriter implements
Visitor<FilterPredicate> {
} else if (and.getRight() instanceof Contains) {
right = and.getRight();
} else {
- return and;
+ right = and.getRight();
}
- if (left instanceof Contains) {
- if (!(right instanceof Contains)) {
- throw new UnsupportedOperationException(
- "Contains predicates cannot be composed with non-Contains
predicates");
- }
+ // If two Contains predicates refer to the same column, optimize by
combining into a single predicate
+ if ((left instanceof Contains && right instanceof Contains)
+ && ((Contains<?>) left)
+ .getColumn()
+ .getColumnPath()
+ .equals(((Contains<?>) right).getColumn().getColumnPath())) {
return ((Contains) left).and(right);
+ } else if (left != and.getLeft() || right != and.getRight()) {
+ return new And(left, right);
} else {
return and;
}
@@ -142,7 +145,7 @@ public final class ContainsRewriter implements
Visitor<FilterPredicate> {
} else if (or.getLeft() instanceof Contains) {
left = or.getLeft();
} else {
- return or;
+ left = or.getLeft();
}
final FilterPredicate right;
@@ -153,15 +156,18 @@ public final class ContainsRewriter implements
Visitor<FilterPredicate> {
} else if (or.getRight() instanceof Contains) {
right = or.getRight();
} else {
- return or;
+ right = or.getRight();
}
- if (left instanceof Contains) {
- if (!(right instanceof Contains)) {
- throw new UnsupportedOperationException(
- "Contains predicates cannot be composed with non-Contains
predicates");
- }
+ // If two Contains predicates refer to the same column, optimize by
combining into a single predicate
+ if ((left instanceof Contains && right instanceof Contains)
+ && ((Contains<?>) left)
+ .getColumn()
+ .getColumnPath()
+ .equals(((Contains<?>) right).getColumn().getColumnPath())) {
return ((Contains) left).or(right);
+ } else if (left != or.getLeft() || right != or.getRight()) {
+ return new Or(left, right);
} else {
return or;
}
diff --git
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/LogicalInverter.java
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/LogicalInverter.java
index d1d006ccf..506b8f0e5 100644
---
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/LogicalInverter.java
+++
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/LogicalInverter.java
@@ -95,7 +95,7 @@ public final class LogicalInverter implements
Visitor<FilterPredicate> {
@Override
public <T extends Comparable<T>> FilterPredicate visit(Contains<T> contains)
{
- throw new UnsupportedOperationException("Contains not supported yet");
+ return contains.not();
}
@Override
diff --git
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
index 474862d02..60dc80cd7 100644
---
a/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
+++
b/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/Operators.java
@@ -25,6 +25,7 @@ import java.util.Locale;
import java.util.Objects;
import java.util.Set;
import java.util.function.BiFunction;
+import java.util.function.Function;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.parquet.io.api.Binary;
@@ -323,6 +324,34 @@ public final class Operators {
}
}
+ private static class DoesNotContain<T extends Comparable<T>> extends
Contains<T> {
+ Contains<T> underlying;
+
+ protected DoesNotContain(Contains<T> underlying) {
+ super(underlying.getColumn());
+ this.underlying = underlying;
+ }
+
+ @Override
+ public <R> R accept(Visitor<R> visitor) {
+ return visitor.visit(this);
+ }
+
+ @Override
+ public <R> R filter(
+ Visitor<R> visitor,
+ BiFunction<R, R, R> andBehavior,
+ BiFunction<R, R, R> orBehavior,
+ Function<R, R> notBehavior) {
+ return notBehavior.apply(visitor.visit(underlying));
+ }
+
+ @Override
+ public String toString() {
+ return "not(" + underlying.toString() + ")";
+ }
+ }
+
public abstract static class Contains<T extends Comparable<T>> implements
FilterPredicate, Serializable {
private final Column<T> column;
@@ -345,11 +374,14 @@ public final class Operators {
}
/**
- * Applies a filtering Vistitor to the Contains predicate, traversing any
composed And or Or clauses,
- * and finally delegating to the underlying ColumnFilterPredicate.
+ * Applies a filtering Visitor to the Contains predicate, traversing any
composed And or Or clauses,
+ * and finally delegating to the underlying column predicate.
*/
public abstract <R> R filter(
- Visitor<R> visitor, BiFunction<R, R, R> andBehavior, BiFunction<R, R,
R> orBehavior);
+ Visitor<R> visitor,
+ BiFunction<R, R, R> andBehavior,
+ BiFunction<R, R, R> orBehavior,
+ Function<R, R> notBehavior);
Contains<T> and(FilterPredicate other) {
return new ContainsComposedPredicate<>(this, (Contains<T>) other,
ContainsComposedPredicate.Combinator.AND);
@@ -358,6 +390,10 @@ public final class Operators {
Contains<T> or(FilterPredicate other) {
return new ContainsComposedPredicate<>(this, (Contains<T>) other,
ContainsComposedPredicate.Combinator.OR);
}
+
+ Contains<T> not() {
+ return new DoesNotContain<>(this);
+ }
}
private static class ContainsComposedPredicate<T extends Comparable<T>>
extends Contains<T> {
@@ -390,9 +426,13 @@ public final class Operators {
}
@Override
- public <R> R filter(Visitor<R> visitor, BiFunction<R, R, R> andBehavior,
BiFunction<R, R, R> orBehavior) {
- final R filterLeft = left.filter(visitor, andBehavior, orBehavior);
- final R filterRight = right.filter(visitor, andBehavior, orBehavior);
+ public <R> R filter(
+ Visitor<R> visitor,
+ BiFunction<R, R, R> andBehavior,
+ BiFunction<R, R, R> orBehavior,
+ Function<R, R> notBehavior) {
+ final R filterLeft = left.filter(visitor, andBehavior, orBehavior,
notBehavior);
+ final R filterRight = right.filter(visitor, andBehavior, orBehavior,
notBehavior);
if (combinator == Combinator.AND) {
return andBehavior.apply(filterLeft, filterRight);
@@ -456,7 +496,11 @@ public final class Operators {
}
@Override
- public <R> R filter(Visitor<R> visitor, BiFunction<R, R, R> andBehavior,
BiFunction<R, R, R> orBehavior) {
+ public <R> R filter(
+ Visitor<R> visitor,
+ BiFunction<R, R, R> andBehavior,
+ BiFunction<R, R, R> orBehavior,
+ Function<R, R> notBehavior) {
return underlying.accept(visitor);
}
}
diff --git
a/parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
b/parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
index 3c28ba6af..c2aab2b6b 100644
---
a/parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
+++
b/parquet-column/src/main/java/org/apache/parquet/filter2/recordlevel/IncrementallyUpdatedFilterPredicate.java
@@ -18,6 +18,7 @@
*/
package org.apache.parquet.filter2.recordlevel;
+import java.util.Arrays;
import java.util.Objects;
import org.apache.parquet.io.api.Binary;
@@ -145,6 +146,83 @@ public interface IncrementallyUpdatedFilterPredicate {
}
}
+ /**
+ * A ValueInspector implementation that keeps state for one or more delegate
inspectors.
+ */
+ abstract static class DelegatingValueInspector extends ValueInspector {
+ private final Iterable<ValueInspector> delegates;
+
+ DelegatingValueInspector(ValueInspector... delegates) {
+ this.delegates = Arrays.asList(delegates);
+ }
+
+ /**
+ * Hook called after every value update. Can update state and set result
on this ValueInspector.
+ */
+ abstract void onUpdate();
+
+ /**
+ * Hook called after updateNull(), if no values have been recorded for the
delegate inspectors.
+ */
+ abstract void onNull();
+
+ Iterable<ValueInspector> getDelegates() {
+ return delegates;
+ }
+
+ @Override
+ public void updateNull() {
+ for (ValueInspector delegate : delegates) {
+ if (!delegate.isKnown()) {
+ delegate.updateNull();
+ }
+ }
+ onNull();
+ }
+
+ @Override
+ public void update(int value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void update(long value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void update(boolean value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void update(float value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void update(double value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void update(Binary value) {
+ delegates.forEach(d -> d.update(value));
+ onUpdate();
+ }
+
+ @Override
+ public void reset() {
+ delegates.forEach(ValueInspector::reset);
+ super.reset();
+ }
+ }
+
// base class for and / or
abstract static class BinaryLogical implements
IncrementallyUpdatedFilterPredicate {
private final IncrementallyUpdatedFilterPredicate left;
diff --git
a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
index f4fe80ab9..ffbb82197 100644
---
a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
+++
b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
@@ -371,7 +371,11 @@ public abstract class ColumnIndexBuilder {
@Override
public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(Contains<T>
contains) {
- return contains.filter(this, IndexIterator::intersection,
IndexIterator::union);
+ return contains.filter(
+ this,
+ IndexIterator::intersection,
+ IndexIterator::union,
+ indices -> IndexIterator.all(getPageCount()));
}
@Override
diff --git
a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
index e46673f01..8b6ee1f95 100644
---
a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
+++
b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
@@ -158,7 +158,7 @@ public class ColumnIndexFilter implements
Visitor<RowRanges> {
@Override
public <T extends Comparable<T>> RowRanges visit(Contains<T> contains) {
- return contains.filter(this, RowRanges::intersection, RowRanges::union);
+ return contains.filter(this, RowRanges::intersection, RowRanges::union,
ranges -> allRows());
}
@Override
diff --git
a/parquet-column/src/test/java/org/apache/parquet/filter2/predicate/TestFilterApiMethods.java
b/parquet-column/src/test/java/org/apache/parquet/filter2/predicate/TestFilterApiMethods.java
index c2e1ef385..9bb180087 100644
---
a/parquet-column/src/test/java/org/apache/parquet/filter2/predicate/TestFilterApiMethods.java
+++
b/parquet-column/src/test/java/org/apache/parquet/filter2/predicate/TestFilterApiMethods.java
@@ -94,20 +94,33 @@ public class TestFilterApiMethods {
}
@Test
- public void testInvalidContainsCreation() {
+ public void testContainsCreation() {
assertThrows(
"Contains predicate does not support null element value",
IllegalArgumentException.class,
() -> contains(eq(binColumn, null)));
+ // Assert that a single Contains predicate referencing multiple columns
throws an error
assertThrows(
"Composed Contains predicates must reference the same column name;
found [a.b.c, b.c.d]",
IllegalArgumentException.class,
- () -> ContainsRewriter.rewrite(or(
- contains(eq(binaryColumn("a.b.c"), Binary.fromString("foo"))),
- and(
- contains(eq(binaryColumn("b.c.d"), Binary.fromString("bar"))),
- contains(eq(binaryColumn("b.c.d"),
Binary.fromString("bar")))))));
+ () -> contains(eq(binaryColumn("a.b.c"), Binary.fromString("foo")))
+ .and(contains(eq(binaryColumn("b.c.d"), Binary.fromString("bar"))))
+ .and(contains(eq(binaryColumn("b.c.d"),
Binary.fromString("bar")))));
+
+ // Assert that a Contains predicate referencing multiple columns is
allowed when composed with and() or or()
+ final FilterPredicate rewritten = ContainsRewriter.rewrite(or(
+ contains(eq(binaryColumn("a.b.c"), Binary.fromString("foo"))),
+ and(
+ contains(eq(binaryColumn("b.c.d"), Binary.fromString("bar"))),
+ contains(eq(binaryColumn("b.c.d"), Binary.fromString("baz"))))));
+ assertTrue(rewritten instanceof Or);
+
+ // Assert that the predicates for column b.c.d have been combined into a
single Contains predicate,
+ // while the predicate for column a.b.c is separate
+ final Or or = (Or) rewritten;
+ assertEquals(binaryColumn("a.b.c"), ((Operators.Contains)
or.getLeft()).getColumn());
+ assertEquals(binaryColumn("b.c.d"), ((Operators.Contains)
or.getRight()).getColumn());
}
@Test
@@ -122,9 +135,9 @@ public class TestFilterApiMethods {
contains(eq(binColumn, Binary.fromString("foo"))),
and(
contains(eq(binColumn, Binary.fromString("bar"))),
- contains(eq(binColumn, Binary.fromString("baz"))))));
+ not(contains(eq(binColumn, Binary.fromString("baz")))))));
assertEquals(
- "or(contains(eq(a.string.column, Binary{\"foo\"})),
and(contains(eq(a.string.column, Binary{\"bar\"})),
contains(eq(a.string.column, Binary{\"baz\"}))))",
+ "or(contains(eq(a.string.column, Binary{\"foo\"})),
and(contains(eq(a.string.column, Binary{\"bar\"})),
not(contains(eq(a.string.column, Binary{\"baz\"})))))",
pred.toString());
}
diff --git
a/parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java
b/parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java
index b356c0ba9..7f66ce382 100644
---
a/parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java
+++
b/parquet-generator/src/main/java/org/apache/parquet/filter2/IncrementallyUpdatedFilterPredicateGenerator.java
@@ -66,6 +66,7 @@ public class IncrementallyUpdatedFilterPredicateGenerator {
public void run() throws IOException {
add("package org.apache.parquet.filter2.recordlevel;\n" + "\n"
+ + "import java.util.Iterator;\n"
+ "import java.util.List;\n"
+ "import java.util.Set;\n"
+ "\n"
@@ -85,6 +86,7 @@ public class IncrementallyUpdatedFilterPredicateGenerator {
+ "import
org.apache.parquet.filter2.predicate.Operators.UserDefined;\n"
+ "import org.apache.parquet.filter2.predicate.UserDefinedPredicate;\n"
+ "import
org.apache.parquet.filter2.recordlevel.IncrementallyUpdatedFilterPredicate.ValueInspector;\n"
+ + "import
org.apache.parquet.filter2.recordlevel.IncrementallyUpdatedFilterPredicate.DelegatingValueInspector;\n"
+ "import org.apache.parquet.io.api.Binary;\n"
+ "import org.apache.parquet.io.PrimitiveColumnIO;\n"
+ "import org.apache.parquet.schema.PrimitiveComparator;\n\n"
@@ -336,17 +338,9 @@ public class IncrementallyUpdatedFilterPredicateGenerator {
+ "\n");
}
- private void addContainsUpdateCase(TypeInfo info, String... inspectors)
throws IOException {
- add(" @Override\n" + " public void update(" + info.primitiveName + "
value) {\n");
- for (String inspector : inspectors) {
- add(" " + inspector + ".update(value);\n");
- }
- add(" checkSatisfied();\n" + " }\n");
- }
-
private void addContainsInspectorVisitor(String op) throws IOException {
add(" @Override\n"
- + " public <T extends Comparable<T>> ValueInspector visit(" + op +
"<T> pred) {\n"
+ + " public <T extends Comparable<T>> ContainsPredicate visit(" + op
+ "<T> pred) {\n"
+ " ColumnPath columnPath = pred.getColumn().getColumnPath();\n"
+ " Class<T> clazz = pred.getColumn().getColumnType();\n"
+ " ValueInspector valueInspector = null;\n");
@@ -382,116 +376,129 @@ public class
IncrementallyUpdatedFilterPredicateGenerator {
}
}
- add(" return valueInspector;" + " }\n");
+ add(" return new ContainsSinglePredicate(valueInspector, false);\n" +
" }\n");
}
private void addContainsBegin() throws IOException {
- add(" private static class ContainsPredicate extends ValueInspector {\n"
- + " private final ValueInspector inspector;\n"
+ add(" private abstract static class ContainsPredicate extends
DelegatingValueInspector {\n"
+ + " ContainsPredicate(ValueInspector... delegates) {\n"
+ + " super(delegates);\n"
+ + " }\n"
+ "\n"
- + " private ContainsPredicate(ValueInspector inspector) {\n"
- + " this.inspector = inspector;\n"
+ + " abstract ContainsPredicate not();\n"
+ + " }\n"
+ + "\n"
+ + " private static class ContainsSinglePredicate extends
ContainsPredicate {\n"
+ + " private final boolean isNot;\n"
+ + " \n"
+ + " private ContainsSinglePredicate(ValueInspector inspector,
boolean isNot) {\n"
+ + " super(inspector);\n"
+ + " this.isNot = isNot;\n"
+ + " }\n\n"
+ + " @Override\n"
+ + " ContainsPredicate not() {\n"
+ + " return new
ContainsSinglePredicate(getDelegates().iterator().next(), true);\n"
+ " }\n"
+ "\n"
- + " private void checkSatisfied() {\n"
- + " if (!isKnown() && inspector.isKnown() &&
inspector.getResult()) {\n"
- + " setResult(true);\n"
+ + " @Override\n"
+ + " void onUpdate() {\n"
+ + " if (isKnown()) {\n"
+ + " return;\n"
+ + " }\n"
+ + "\n"
+ + " for (ValueInspector inspector : getDelegates()) {\n"
+ + " if (inspector.isKnown() && inspector.getResult()) {\n"
+ + " setResult(!isNot);\n"
+ + " return;\n"
+ + " }\n"
+ " }\n"
+ " }\n"
+ "\n"
+ " @Override\n"
- + " public void updateNull() {\n"
- + " setResult(false);\n"
- + " }\n");
-
- for (TypeInfo info : TYPES) {
- addContainsUpdateCase(info, "inspector");
- }
-
- add(" @Override\n" + " public void reset() {\n"
- + " super.reset();\n"
- + " inspector.reset();\n"
+ + " void onNull() {\n"
+ + " setResult(isNot);\n"
+ " }\n"
- + " }\n");
+ + " }\n\n");
- add(" private static class ContainsAndPredicate extends ValueInspector
{\n"
- + " private final ValueInspector left;\n"
- + " private final ValueInspector right;\n"
- + "\n"
- + " private ContainsAndPredicate(ValueInspector left,
ValueInspector right) {\n"
- + " this.left = left;\n"
- + " this.right = right;\n"
+ add(" private static class ContainsAndPredicate extends ContainsPredicate
{\n"
+ + " private ContainsAndPredicate(ContainsPredicate left,
ContainsPredicate right) {\n"
+ + " super(left, right);\n"
+ " }\n"
+ "\n"
- + " private void checkSatisfied() {\n"
+ + " @Override\n"
+ + " void onUpdate() {\n"
+ " if (isKnown()) { return; }\n"
- + " if (left.isKnown() && right.isKnown() && left.getResult() &&
right.getResult()) {\n"
+ + "\n"
+ + " boolean allKnown = true;\n"
+ + " for (ValueInspector delegate : getDelegates()) {\n"
+ + " if (delegate.isKnown() && !delegate.getResult()) {\n"
+ + " setResult(false);\n"
+ + " return;\n"
+ + " }\n"
+ + " allKnown = allKnown && delegate.isKnown();\n"
+ + " }\n"
+ + " \n"
+ + " if (allKnown) {\n"
+ " setResult(true);\n"
+ " }\n"
+ " }\n"
- + " \n"
+ + "\n"
+ " @Override\n"
- + " public void updateNull() {\n"
- + " setResult(false);\n"
- + " }\n\n");
-
- for (TypeInfo info : TYPES) {
- addContainsUpdateCase(info, "left", "right");
- }
-
- add(" @Override\n"
- + " public void reset() {\n"
- + " super.reset();\n"
- + " left.reset();\n"
- + " right.reset();\n"
+ + " void onNull() {\n"
+ + " for (ValueInspector delegate : getDelegates()) {\n"
+ + " if (!delegate.getResult()) {\n"
+ + " setResult(false);\n"
+ + " return;\n"
+ + " }\n"
+ + " }\n"
+ + " setResult(true);\n"
+ " }\n"
- + " }\n");
-
- add(" private static class ContainsOrPredicate extends ValueInspector {\n"
- + " private final ValueInspector left;\n"
- + " private final ValueInspector right;\n"
+ "\n"
- + " private ContainsOrPredicate(ValueInspector left, ValueInspector
right) {\n"
- + " this.left = left;\n"
- + " this.right = right;\n"
+ + " @Override\n"
+ + " ContainsPredicate not() {\n"
+ + " Iterator<ValueInspector> it = getDelegates().iterator();\n"
+ + " return new ContainsAndPredicate(((ContainsPredicate)
it.next()).not(), ((ContainsPredicate) it.next()).not());\n"
+ + " }\n"
+ + " }\n\n");
+
+ add(" private static class ContainsOrPredicate extends ContainsPredicate
{\n"
+ + " private ContainsOrPredicate(ContainsPredicate left,
ContainsPredicate right) {\n"
+ + " super(left, right);\n"
+ " }\n"
+ "\n"
- + " private void checkSatisfied() {\n"
+ + " @Override\n"
+ + " void onUpdate() {\n"
+ " if (isKnown()) { return; }\n"
- + " if (left.isKnown() && left.getResult()) {\n"
- + " setResult(true);\n"
- + " return;\n"
- + " }\n"
- + " if (right.isKnown() && right.getResult()) {\n"
- + " setResult(true);\n"
+ + "\n"
+ + " for (ValueInspector delegate : getDelegates()) {\n"
+ + " if (delegate.isKnown() && delegate.getResult()) {\n"
+ + " setResult(true);\n"
+ + " }\n"
+ " }\n"
+ " }\n"
- + " \n"
+ + "\n"
+ " @Override\n"
- + " public void updateNull() {\n"
+ + " void onNull() {\n"
+ + " for (ValueInspector delegate : getDelegates()) {\n"
+ + " if (delegate.getResult()) {\n"
+ + " setResult(true);\n"
+ + " return;\n"
+ + " }\n"
+ + " }\n"
+ " setResult(false);\n"
- + " }\n");
-
- for (TypeInfo info : TYPES) {
- addContainsUpdateCase(info, "left", "right");
- }
-
- add(" @Override\n"
- + " public void reset() {\n"
- + " super.reset();\n"
- + " left.reset();\n"
- + " right.reset();\n"
+ " }\n"
- + " }\n");
+ + "\n"
+ + " @Override\n"
+ + " ContainsPredicate not() {\n"
+ + " Iterator<ValueInspector> it = getDelegates().iterator();\n"
+ + " return new ContainsOrPredicate(((ContainsPredicate)
it.next()).not(), ((ContainsPredicate) it.next()).not());\n"
+ + " }\n"
+ + " }\n\n");
- add(" private class ContainsInspectorVisitor implements
FilterPredicate.Visitor<ValueInspector> {\n\n"
+ add(" private class ContainsInspectorVisitor implements
FilterPredicate.Visitor<ContainsPredicate> {\n\n"
+ " @Override\n"
- + " public <T extends Comparable<T>> ValueInspector
visit(Contains<T> contains) {\n"
- + " return contains.filter(\n"
- + " this,\n"
- + " (l, r) -> new ContainsAndPredicate(l, r),\n"
- + " (l, r) -> new ContainsOrPredicate(l, r)\n"
- + " );\n"
+ + " public <T extends Comparable<T>> ContainsPredicate
visit(Contains<T> contains) {\n"
+ + " return contains.filter(this, ContainsAndPredicate::new,
ContainsOrPredicate::new, ContainsPredicate::not);\n"
+ " }\n");
addContainsInspectorVisitor("Eq");
@@ -504,27 +511,27 @@ public class IncrementallyUpdatedFilterPredicateGenerator
{
addContainsInspectorVisitor("NotIn");
add(" @Override\n"
- + " public ValueInspector visit(Operators.And pred) {\n"
+ + " public ContainsPredicate visit(Operators.And pred) {\n"
+ " throw new UnsupportedOperationException(\"Operators.And not
supported for Contains predicate\");\n"
+ " }\n"
+ "\n"
+ " @Override\n"
- + " public ValueInspector visit(Operators.Or pred) {\n"
+ + " public ContainsPredicate visit(Operators.Or pred) {\n"
+ " throw new UnsupportedOperationException(\"Operators.Or not
supported for Contains predicate\");\n"
+ " }\n"
+ "\n"
+ " @Override\n"
- + " public ValueInspector visit(Operators.Not pred) {\n"
+ + " public ContainsPredicate visit(Operators.Not pred) {\n"
+ " throw new UnsupportedOperationException(\"Operators.Not not
supported for Contains predicate\");\n"
+ " }"
+ " @Override\n"
- + " public <T extends Comparable<T>, U extends
UserDefinedPredicate<T>> ValueInspector visit(\n"
+ + " public <T extends Comparable<T>, U extends
UserDefinedPredicate<T>> ContainsPredicate visit(\n"
+ " UserDefined<T, U> pred) {\n"
+ " throw new
UnsupportedOperationException(\"UserDefinedPredicate not supported for Contains
predicate\");\n"
+ " }\n"
+ "\n"
+ " @Override\n"
- + " public <T extends Comparable<T>, U extends
UserDefinedPredicate<T>> ValueInspector visit(\n"
+ + " public <T extends Comparable<T>, U extends
UserDefinedPredicate<T>> ContainsPredicate visit(\n"
+ " LogicalNotUserDefined<T, U> pred) {\n"
+ " throw new
UnsupportedOperationException(\"LogicalNotUserDefined not supported for
Contains predicate\");\n"
+ " }\n"
@@ -533,7 +540,7 @@ public class IncrementallyUpdatedFilterPredicateGenerator {
}
private void addContainsCase() throws IOException {
- add(" valueInspector = new ContainsPredicate(new
ContainsInspectorVisitor().visit(pred));\n");
+ add(" valueInspector = new ContainsInspectorVisitor().visit(pred);\n");
}
private void addContainsEnd() {
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java
index 16348e535..39babc0ac 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java
@@ -121,7 +121,7 @@ public class BloomFilterImpl implements
FilterPredicate.Visitor<Boolean> {
@Override
public <T extends Comparable<T>> Boolean visit(Operators.Contains<T>
contains) {
- return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r);
+ return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
@Override
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java
index dbb38047e..be4455eeb 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java
@@ -490,7 +490,7 @@ public class DictionaryFilter implements
FilterPredicate.Visitor<Boolean> {
@Override
public <T extends Comparable<T>> Boolean visit(Contains<T> contains) {
- return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r);
+ return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
@Override
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java
index deb4706d5..4d7918c4f 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/statisticslevel/StatisticsFilter.java
@@ -214,7 +214,7 @@ public class StatisticsFilter implements
FilterPredicate.Visitor<Boolean> {
@Override
public <T extends Comparable<T>> Boolean visit(Contains<T> contains) {
- return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r);
+ return contains.filter(this, (l, r) -> l || r, (l, r) -> l && r, v ->
BLOCK_MIGHT_MATCH);
}
@Override
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/filter2/recordlevel/TestRecordLevelFilters.java
b/parquet-hadoop/src/test/java/org/apache/parquet/filter2/recordlevel/TestRecordLevelFilters.java
index 888f2d052..1a1a31e73 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/filter2/recordlevel/TestRecordLevelFilters.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/filter2/recordlevel/TestRecordLevelFilters.java
@@ -46,6 +46,7 @@ import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
+import java.util.stream.LongStream;
import org.apache.parquet.example.data.Group;
import org.apache.parquet.filter2.compat.FilterCompat;
import org.apache.parquet.filter2.predicate.FilterPredicate;
@@ -106,6 +107,8 @@ public class TestRecordLevelFilters {
users.add(new User(30, null, Arrays.asList(new PhoneNumber(1111111111L,
"home")), null));
+ users.add(new User(31, null, Arrays.asList(new PhoneNumber(2222222222L,
"business")), null));
+
for (int i = 100; i < 200; i++) {
Location location = null;
if (i % 3 == 1) {
@@ -222,11 +225,15 @@ public class TestRecordLevelFilters {
contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))), 27L, 28L, 30L);
assertPredicate(
- contains(notEq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell"))), 27L, 28L, 30L);
+ contains(notEq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell"))),
+ 27L,
+ 28L,
+ 30L,
+ 31L);
- assertPredicate(contains(gt(longColumn("phoneNumbers.phone.number"),
1111111111L)), 20L, 27L, 28L);
+ assertPredicate(contains(gt(longColumn("phoneNumbers.phone.number"),
1111111111L)), 20L, 27L, 28L, 31L);
- assertPredicate(contains(gtEq(longColumn("phoneNumbers.phone.number"),
1111111111L)), 20L, 27L, 28L, 30L);
+ assertPredicate(contains(gtEq(longColumn("phoneNumbers.phone.number"),
1111111111L)), 20L, 27L, 28L, 30L, 31L);
assertPredicate(contains(lt(longColumn("phoneNumbers.phone.number"),
105L)), 100L, 101L, 102L, 103L, 104L);
@@ -245,7 +252,86 @@ public class TestRecordLevelFilters {
contains(notIn(binaryColumn("phoneNumbers.phone.kind"),
ImmutableSet.of(Binary.fromString("cell")))),
27L,
28L,
- 30L);
+ 30L,
+ 31L);
+ }
+
+ @Test
+ public void testArrayDoesNotContains() throws Exception {
+ assertPredicate(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ 17L,
+ 18L,
+ 19L,
+ 20L,
+ 28L,
+ 30L,
+ 31L);
+
+ // test composed not(contains())
+ assertPredicate(
+ and(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))))),
+ 17L,
+ 18L,
+ 19L,
+ 20L,
+ 31L);
+
+ assertPredicate(
+ and(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ and(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home")))),
+ not(contains(
+ eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("business")))))),
+ 17L,
+ 18L,
+ 19L,
+ 20L);
+
+ assertPredicate(
+ or(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))))),
+ LongStream.concat(LongStream.of(17L, 18L, 19L, 20L, 28L, 30L, 31L),
LongStream.range(100L, 200L))
+ .toArray());
+
+ assertPredicate(
+ or(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ and(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home")))),
+ not(contains(
+ eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("mobile")))))),
+ LongStream.concat(LongStream.of(17L, 18L, 19L, 20L, 28L, 30L, 31L),
LongStream.range(100L, 200L))
+ .toArray());
+
+ // Test composed contains() with not(contains())
+ assertPredicate(
+ and(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ or(
+ contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))),
+ contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("business"))))),
+ 28L,
+ 30L,
+ 31L);
+
+ assertPredicate(
+ or(
+ not(contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("cell")))),
+ and(
+ contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))),
+ contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("apartment"))))),
+ 17L,
+ 18L,
+ 19L,
+ 20L,
+ 28L,
+ 30L,
+ 31L);
}
@Test
@@ -282,7 +368,8 @@ public class TestRecordLevelFilters {
contains(eq(longColumn("phoneNumbers.phone.number"),
2222222222L))),
20L,
27L,
- 28L);
+ 28L,
+ 31L);
assertPredicate(
or(
@@ -301,7 +388,8 @@ public class TestRecordLevelFilters {
contains(eq(longColumn("phoneNumbers.phone.number"),
2222222222L)))),
20L,
27L,
- 28L);
+ 28L,
+ 31L);
}
@Test
@@ -313,6 +401,15 @@ public class TestRecordLevelFilters {
assertPredicate(contains(eq(doubleColumn("accounts.key_value.value"),
1000.0D)), 17L, 19L);
}
+ @Test
+ public void testArrayContainsMixedColumns() throws Exception {
+ assertPredicate(
+ and(
+ contains(eq(binaryColumn("phoneNumbers.phone.kind"),
Binary.fromString("home"))),
+ not(contains(eq(longColumn("phoneNumbers.phone.number"),
2222222222L)))),
+ 30L);
+ }
+
@Test
public void testNameNotNull() throws Exception {
BinaryColumn name = binaryColumn("name");