diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/And.java b/paimon-api/src/main/java/org/apache/paimon/predicate/And.java similarity index 70% rename from paimon-common/src/main/java/org/apache/paimon/predicate/And.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/And.java index 372e2c501fd3..5cdae3aed9fe 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/And.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/And.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; - import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -34,31 +31,6 @@ public class And extends CompoundPredicate.Function { private And() {} - @Override - public boolean test(InternalRow row, List children) { - for (Predicate child : children) { - if (!child.test(row)) { - return false; - } - } - return true; - } - - @Override - public boolean test( - long rowCount, - InternalRow minValues, - InternalRow maxValues, - InternalArray nullCounts, - List children) { - for (Predicate child : children) { - if (!child.test(rowCount, minValues, maxValues, nullCounts)) { - return false; - } - } - return true; - } - @Override public Optional negate(List children) { List negatedChildren = new ArrayList<>(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/CastTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/CastTransform.java similarity index 63% rename from paimon-common/src/main/java/org/apache/paimon/predicate/CastTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/CastTransform.java index 37141011f309..d4dcba87b86e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/CastTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/CastTransform.java @@ -18,19 +18,13 @@ package org.apache.paimon.predicate; -import org.apache.paimon.casting.CastExecutor; -import org.apache.paimon.casting.CastExecutors; -import org.apache.paimon.data.InternalRow; import org.apache.paimon.types.DataType; -import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; -import static org.apache.paimon.utils.InternalRowUtils.get; - /** Transform that casts a field to a new type. */ public class CastTransform implements Transform { @@ -38,27 +32,21 @@ public class CastTransform implements Transform { private final FieldRef fieldRef; private final DataType type; - private transient CastExecutor cast; - private CastTransform(FieldRef fieldRef, DataType type, CastExecutor cast) { + private CastTransform(FieldRef fieldRef, DataType type) { this.fieldRef = fieldRef; this.type = type; - this.cast = cast; + } + + public static CastTransform of(FieldRef fieldRef, DataType type) { + return new CastTransform(fieldRef, type); } public static Optional tryCreate(FieldRef fieldRef, DataType type) { if (fieldRef.type().equals(type)) { return Optional.of(new FieldTransform(fieldRef)); } - - @SuppressWarnings("unchecked") - CastExecutor cast = - (CastExecutor) CastExecutors.resolve(fieldRef.type(), type); - if (cast == null) { - return Optional.empty(); - } else { - return Optional.of(new CastTransform(fieldRef, type, cast)); - } + return Optional.of(new CastTransform(fieldRef, type)); } @Override @@ -71,15 +59,10 @@ public DataType outputType() { return type; } - @Override - public Object transform(InternalRow row) { - return cast.cast(get(row, fieldRef.index(), fieldRef.type())); - } - @Override public Transform copyWithNewInputs(List inputs) { assert inputs.size() == 1; - return new CastTransform((FieldRef) inputs.get(0), type, cast); + return new CastTransform((FieldRef) inputs.get(0), type); } @Override @@ -100,13 +83,4 @@ public int hashCode() { public String toString() { return "CAST( " + fieldRef + " AS " + type + ")"; } - - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - @SuppressWarnings("unchecked") - CastExecutor resolved = - (CastExecutor) CastExecutors.resolve(fieldRef.type(), type); - this.cast = resolved; - } } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/CompareUtils.java b/paimon-api/src/main/java/org/apache/paimon/predicate/CompareUtils.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/CompareUtils.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/CompareUtils.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java b/paimon-api/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java similarity index 79% rename from paimon-common/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java index 7b880c8a9d90..6bd1cc5b0e16 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/CompoundPredicate.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; - import java.io.Serializable; import java.util.List; import java.util.Objects; @@ -48,17 +45,6 @@ public List children() { return children; } - @Override - public boolean test(InternalRow row) { - return function.test(row, children); - } - - @Override - public boolean test( - long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts) { - return function.test(rowCount, minValues, maxValues, nullCounts, children); - } - @Override public Optional negate() { return function.negate(children); @@ -91,15 +77,6 @@ public String toString() { /** Evaluate the predicate result based on multiple {@link Predicate}s. */ public abstract static class Function implements Serializable { - public abstract boolean test(InternalRow row, List children); - - public abstract boolean test( - long rowCount, - InternalRow minValues, - InternalRow maxValues, - InternalArray nullCounts, - List children); - public abstract Optional negate(List children); public abstract T visit(FunctionVisitor visitor, List children); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/ConcatTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/ConcatTransform.java similarity index 87% rename from paimon-common/src/main/java/org/apache/paimon/predicate/ConcatTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/ConcatTransform.java index 40d51f075fd4..2d0cf831fd6a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/ConcatTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/ConcatTransform.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; - import java.util.List; /** Concat {@link Transform}. */ @@ -31,11 +29,6 @@ public ConcatTransform(List inputs) { super(inputs); } - @Override - public BinaryString transform(List inputs) { - return BinaryString.concat(inputs); - } - @Override public Transform copyWithNewInputs(List inputs) { return new ConcatTransform(inputs); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java similarity index 83% rename from paimon-common/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java index 197b1f34f079..a586ec8bf363 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/ConcatWsTransform.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; - import java.util.List; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -34,12 +32,6 @@ public ConcatWsTransform(List inputs) { checkArgument(inputs.size() >= 2); } - @Override - public BinaryString transform(List inputs) { - BinaryString separator = inputs.get(0); - return BinaryString.concatWs(separator, inputs.subList(1, inputs.size())); - } - @Override public Transform copyWithNewInputs(List inputs) { return new ConcatWsTransform(inputs); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Contains.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Contains.java similarity index 72% rename from paimon-common/src/main/java/org/apache/paimon/predicate/Contains.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/Contains.java index bf97e24471fd..1e6d07eada89 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Contains.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Contains.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; @@ -31,23 +28,6 @@ public class Contains extends NullFalseLeafBinaryFunction { private Contains() {} - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.contains((BinaryString) patternLiteral); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - Object patternLiteral) { - return true; - } - @Override public Optional negate() { return Optional.empty(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java b/paimon-api/src/main/java/org/apache/paimon/predicate/EndsWith.java similarity index 72% rename from paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/EndsWith.java index 13fcf1249038..e16472bc92cc 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/EndsWith.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; @@ -34,23 +31,6 @@ public class EndsWith extends NullFalseLeafBinaryFunction { private EndsWith() {} - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.endsWith((BinaryString) patternLiteral); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - Object patternLiteral) { - return true; - } - @Override public Optional negate() { return Optional.empty(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Equal.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Equal.java similarity index 73% rename from paimon-common/src/main/java/org/apache/paimon/predicate/Equal.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/Equal.java index e826fb0f1e3f..4bbe53c9a955 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Equal.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Equal.java @@ -18,13 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - /** A {@link NullFalseLeafBinaryFunction} to eval equal. */ public class Equal extends NullFalseLeafBinaryFunction { @@ -32,17 +28,6 @@ public class Equal extends NullFalseLeafBinaryFunction { private Equal() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) == 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, min) >= 0 && compareLiteral(type, literal, max) <= 0; - } - @Override public Optional negate() { return Optional.of(NotEqual.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/FieldRef.java b/paimon-api/src/main/java/org/apache/paimon/predicate/FieldRef.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/FieldRef.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/FieldRef.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/FieldTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/FieldTransform.java similarity index 89% rename from paimon-common/src/main/java/org/apache/paimon/predicate/FieldTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/FieldTransform.java index 5b752950e153..b0325de27839 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/FieldTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/FieldTransform.java @@ -18,15 +18,12 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalRow; import org.apache.paimon.types.DataType; import java.util.Collections; import java.util.List; import java.util.Objects; -import static org.apache.paimon.utils.InternalRowUtils.get; - /** Transform that extracts a field from a row. */ public class FieldTransform implements Transform { @@ -52,11 +49,6 @@ public DataType outputType() { return fieldRef.type(); } - @Override - public Object transform(InternalRow row) { - return get(row, fieldRef.index(), fieldRef.type()); - } - @Override public Transform copyWithNewInputs(List inputs) { assert inputs.size() == 1; diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java b/paimon-api/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java similarity index 75% rename from paimon-common/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java index d4ec8639b463..4dacae12200f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/GreaterOrEqual.java @@ -18,13 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - /** A {@link NullFalseLeafBinaryFunction} to eval greater or equal. */ public class GreaterOrEqual extends NullFalseLeafBinaryFunction { @@ -32,17 +28,6 @@ public class GreaterOrEqual extends NullFalseLeafBinaryFunction { private GreaterOrEqual() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) <= 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, max) <= 0; - } - @Override public Optional negate() { return Optional.of(LessThan.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/GreaterThan.java b/paimon-api/src/main/java/org/apache/paimon/predicate/GreaterThan.java similarity index 75% rename from paimon-common/src/main/java/org/apache/paimon/predicate/GreaterThan.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/GreaterThan.java index 0b498b3ab066..1de7651d9b8b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/GreaterThan.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/GreaterThan.java @@ -18,13 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - /** A {@link LeafFunction} to eval greater. */ public class GreaterThan extends NullFalseLeafBinaryFunction { @@ -32,17 +28,6 @@ public class GreaterThan extends NullFalseLeafBinaryFunction { private GreaterThan() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) < 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, max) < 0; - } - @Override public Optional negate() { return Optional.of(LessOrEqual.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java b/paimon-api/src/main/java/org/apache/paimon/predicate/In.java similarity index 58% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/In.java index 575ef8371dda..d3be62ab4882 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/In.java @@ -18,33 +18,25 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; +import java.util.Optional; -/** Function to test a field. */ -public abstract class LeafUnaryFunction extends LeafFunction { +/** A {@link LeafFunction} to eval in. */ +public class In extends LeafFunction { private static final long serialVersionUID = 1L; - public abstract boolean test(DataType type, Object value); + public static final In INSTANCE = new In(); - public abstract boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount); + private In() {} @Override - public boolean test(DataType type, Object value, List literals) { - return test(type, value); + public Optional negate() { + return Optional.of(NotIn.INSTANCE); } @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals) { - return test(type, rowCount, min, max, nullCount); + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitIn(fieldRef, literals); } } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/InPredicateVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/InPredicateVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/InPredicateVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/InPredicateVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/IsNotNull.java b/paimon-api/src/main/java/org/apache/paimon/predicate/IsNotNull.java similarity index 81% rename from paimon-common/src/main/java/org/apache/paimon/predicate/IsNotNull.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/IsNotNull.java index 3d2f9584ef35..baa33b71a75b 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/IsNotNull.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/IsNotNull.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; @@ -30,16 +28,6 @@ public class IsNotNull extends LeafUnaryFunction { private IsNotNull() {} - @Override - public boolean test(DataType type, Object field) { - return field != null; - } - - @Override - public boolean test(DataType type, long rowCount, Object min, Object max, Long nullCount) { - return nullCount == null || nullCount < rowCount; - } - @Override public Optional negate() { return Optional.of(IsNull.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/IsNull.java b/paimon-api/src/main/java/org/apache/paimon/predicate/IsNull.java similarity index 81% rename from paimon-common/src/main/java/org/apache/paimon/predicate/IsNull.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/IsNull.java index f825b2e2316e..abf8757a2d62 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/IsNull.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/IsNull.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; @@ -30,16 +28,6 @@ public class IsNull extends LeafUnaryFunction { private IsNull() {} - @Override - public boolean test(DataType type, Object field) { - return field == null; - } - - @Override - public boolean test(DataType type, long rowCount, Object min, Object max, Long nullCount) { - return nullCount == null || nullCount > 0; - } - @Override public Optional negate() { return Optional.of(IsNotNull.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafFunction.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafFunction.java similarity index 82% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LeafFunction.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/LeafFunction.java index 7a1e3d0f5f51..d205499dc72a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafFunction.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafFunction.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.io.Serializable; import java.util.List; import java.util.Optional; @@ -27,16 +25,6 @@ /** Function to test a field with literals. */ public abstract class LeafFunction implements Serializable { - public abstract boolean test(DataType type, Object field, List literals); - - public abstract boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals); - public abstract Optional negate(); @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafPredicate.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafPredicate.java similarity index 63% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LeafPredicate.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/LeafPredicate.java index 6be94495564c..aa41f9eeb524 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafPredicate.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafPredicate.java @@ -18,25 +18,13 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.serializer.InternalSerializers; -import org.apache.paimon.data.serializer.ListSerializer; -import org.apache.paimon.data.serializer.NullableSerializer; -import org.apache.paimon.io.DataInputViewStreamWrapper; -import org.apache.paimon.io.DataOutputViewStreamWrapper; import org.apache.paimon.types.DataType; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Optional; -import static org.apache.paimon.utils.InternalRowUtils.get; - /** Leaf node of a {@link Predicate} tree. Compares a field in the row with literals. */ public class LeafPredicate extends TransformPredicate { @@ -88,23 +76,6 @@ public LeafPredicate copyWithNewIndex(int fieldIndex) { return new LeafPredicate(function, type(), fieldIndex, fieldName(), literals); } - @Override - public boolean test( - long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts) { - Object min = get(minValues, index(), type()); - Object max = get(maxValues, index(), type()); - Long nullCount = nullCounts.isNullAt(index()) ? null : nullCounts.getLong(index()); - if (nullCount == null || rowCount != nullCount) { - // not all null - // min or max is null - // unknown stats - if (min == null || max == null) { - return true; - } - } - return function.test(type(), rowCount, min, max, nullCount, literals); - } - @Override public Optional negate() { return function.negate() @@ -130,19 +101,4 @@ public String toString() { ? function + "(" + fieldName() + ")" : function + "(" + fieldName() + ", " + literalsStr + ")"; } - - private ListSerializer objectsSerializer() { - return new ListSerializer<>( - NullableSerializer.wrapIfNullIsNotSupported(InternalSerializers.create(type()))); - } - - private void writeObject(ObjectOutputStream out) throws IOException { - out.defaultWriteObject(); - objectsSerializer().serialize(literals, new DataOutputViewStreamWrapper(out)); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - literals = objectsSerializer().deserialize(new DataInputViewStreamWrapper(in)); - } } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LeafPredicateExtractor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafPredicateExtractor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LeafPredicateExtractor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/LeafPredicateExtractor.java diff --git a/paimon-api/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java new file mode 100644 index 000000000000..9b46f6067620 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/LeafUnaryFunction.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.predicate; + +/** Marker {@link LeafFunction} for unary predicates (no literals). */ +public abstract class LeafUnaryFunction extends LeafFunction { + + private static final long serialVersionUID = 1L; +} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LessOrEqual.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LessOrEqual.java similarity index 75% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LessOrEqual.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/LessOrEqual.java index cc17decad15b..c61505eb52f4 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/LessOrEqual.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/LessOrEqual.java @@ -18,13 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - /** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ public class LessOrEqual extends NullFalseLeafBinaryFunction { @@ -32,17 +28,6 @@ public class LessOrEqual extends NullFalseLeafBinaryFunction { private LessOrEqual() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) >= 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, min) >= 0; - } - @Override public Optional negate() { return Optional.of(GreaterThan.INSTANCE); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/LessThan.java b/paimon-api/src/main/java/org/apache/paimon/predicate/LessThan.java similarity index 71% rename from paimon-common/src/main/java/org/apache/paimon/predicate/LessThan.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/LessThan.java index e26c8d68cef9..29f303412e41 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/LessThan.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/LessThan.java @@ -18,31 +18,16 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - -/** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ +/** A {@link NullFalseLeafBinaryFunction} to eval less than. */ public class LessThan extends NullFalseLeafBinaryFunction { public static final LessThan INSTANCE = new LessThan(); private LessThan() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) > 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, min) > 0; - } - @Override public Optional negate() { return Optional.of(GreaterOrEqual.INSTANCE); diff --git a/paimon-api/src/main/java/org/apache/paimon/predicate/Like.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Like.java new file mode 100644 index 000000000000..5c417fca4628 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Like.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.predicate; + +import java.util.List; +import java.util.Optional; + +/** A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like}. */ +public class Like extends NullFalseLeafBinaryFunction { + + public static final Like INSTANCE = new Like(); + + private Like() {} + + @Override + public Optional negate() { + return Optional.empty(); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitLike(fieldRef, literals.get(0)); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/NotEqual.java b/paimon-api/src/main/java/org/apache/paimon/predicate/NotEqual.java similarity index 73% rename from paimon-common/src/main/java/org/apache/paimon/predicate/NotEqual.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/NotEqual.java index a3bdfbb992f9..f1e404e41c02 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/NotEqual.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/NotEqual.java @@ -18,13 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - /** A {@link NullFalseLeafBinaryFunction} to eval not equal. */ public class NotEqual extends NullFalseLeafBinaryFunction { @@ -32,17 +28,6 @@ public class NotEqual extends NullFalseLeafBinaryFunction { private NotEqual() {} - @Override - public boolean test(DataType type, Object field, Object literal) { - return compareLiteral(type, literal, field) != 0; - } - - @Override - public boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal) { - return compareLiteral(type, literal, min) != 0 || compareLiteral(type, literal, max) != 0; - } - @Override public Optional negate() { return Optional.of(Equal.INSTANCE); diff --git a/paimon-api/src/main/java/org/apache/paimon/predicate/NotIn.java b/paimon-api/src/main/java/org/apache/paimon/predicate/NotIn.java new file mode 100644 index 000000000000..ad162f2ba620 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/NotIn.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.predicate; + +import java.util.List; +import java.util.Optional; + +/** A {@link LeafFunction} to eval not in. */ +public class NotIn extends LeafFunction { + + private static final long serialVersionUID = 1L; + + public static final NotIn INSTANCE = new NotIn(); + + private NotIn() {} + + @Override + public Optional negate() { + return Optional.of(In.INSTANCE); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitNotIn(fieldRef, literals); + } +} diff --git a/paimon-api/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java b/paimon-api/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java new file mode 100644 index 000000000000..8873317071a9 --- /dev/null +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.predicate; + +/** Marker {@link LeafFunction} for binary predicates with "null -> false" semantics. */ +public abstract class NullFalseLeafBinaryFunction extends LeafFunction { + + private static final long serialVersionUID = 1L; +} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Or.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Or.java similarity index 70% rename from paimon-common/src/main/java/org/apache/paimon/predicate/Or.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/Or.java index 3385677ab901..fa6f34ba39b8 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Or.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Or.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; - import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -34,31 +31,6 @@ public class Or extends CompoundPredicate.Function { private Or() {} - @Override - public boolean test(InternalRow row, List children) { - for (Predicate child : children) { - if (child.test(row)) { - return true; - } - } - return false; - } - - @Override - public boolean test( - long rowCount, - InternalRow minValues, - InternalRow maxValues, - InternalArray nullCounts, - List children) { - for (Predicate child : children) { - if (child.test(rowCount, minValues, maxValues, nullCounts)) { - return true; - } - } - return false; - } - @Override public Optional negate(List children) { List negatedChildren = new ArrayList<>(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PartitionPredicateVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/PartitionPredicateVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/PartitionPredicateVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/PartitionPredicateVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Predicate.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Predicate.java similarity index 63% rename from paimon-common/src/main/java/org/apache/paimon/predicate/Predicate.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/Predicate.java index b04029ba73ef..cbca768eb6ae 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Predicate.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Predicate.java @@ -19,14 +19,12 @@ package org.apache.paimon.predicate; import org.apache.paimon.annotation.Public; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; import java.io.Serializable; import java.util.Optional; /** - * Predicate which returns Boolean and provides testing by stats. + * A predicate expression. * * @see PredicateBuilder * @since 0.4.0 @@ -34,22 +32,6 @@ @Public public interface Predicate extends Serializable { - /** - * Test based on the specific input row. - * - * @return return true when hit, false when not hit. - */ - boolean test(InternalRow row); - - /** - * Test based on the statistical information to determine whether a hit is possible. - * - * @return return true is likely to hit (there may also be false positives), return false is - * absolutely not possible to hit. - */ - boolean test( - long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts); - /** @return the negation predicate of this predicate if possible. */ Optional negate(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateProjectionConverter.java b/paimon-api/src/main/java/org/apache/paimon/predicate/PredicateProjectionConverter.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/PredicateProjectionConverter.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/PredicateProjectionConverter.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateReplaceVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/PredicateReplaceVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/PredicateReplaceVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/PredicateReplaceVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateVisitor.java b/paimon-api/src/main/java/org/apache/paimon/predicate/PredicateVisitor.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/PredicateVisitor.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/PredicateVisitor.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/SortValue.java b/paimon-api/src/main/java/org/apache/paimon/predicate/SortValue.java similarity index 100% rename from paimon-common/src/main/java/org/apache/paimon/predicate/SortValue.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/SortValue.java diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/StartsWith.java b/paimon-api/src/main/java/org/apache/paimon/predicate/StartsWith.java similarity index 63% rename from paimon-common/src/main/java/org/apache/paimon/predicate/StartsWith.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/StartsWith.java index 56e5e8ecf310..391539eb3804 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/StartsWith.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/StartsWith.java @@ -18,9 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.types.DataType; - import java.util.List; import java.util.Optional; @@ -34,27 +31,6 @@ public class StartsWith extends NullFalseLeafBinaryFunction { private StartsWith() {} - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.startsWith((BinaryString) patternLiteral); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - Object patternLiteral) { - BinaryString minStr = (BinaryString) min; - BinaryString maxStr = (BinaryString) max; - BinaryString pattern = (BinaryString) patternLiteral; - return (minStr.startsWith(pattern) || minStr.compareTo(pattern) <= 0) - && (maxStr.startsWith(pattern) || maxStr.compareTo(pattern) >= 0); - } - @Override public Optional negate() { return Optional.empty(); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/StringTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/StringTransform.java similarity index 75% rename from paimon-common/src/main/java/org/apache/paimon/predicate/StringTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/StringTransform.java index c01d3bcb28e3..8f72d8eab3e0 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/StringTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/StringTransform.java @@ -18,12 +18,9 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.InternalRow; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; -import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -47,7 +44,8 @@ public StringTransform(List inputs) { FieldRef ref = (FieldRef) input; checkArgument(ref.type().is(CHARACTER_STRING)); } else { - checkArgument(input instanceof BinaryString); + // Literal inputs are handled by paimon-common utilities during evaluation. + // Here we only keep the expression structure. } } } @@ -62,23 +60,6 @@ public final DataType outputType() { return DataTypes.STRING(); } - @Override - public final Object transform(InternalRow row) { - List strings = new ArrayList<>(inputs.size()); - for (Object input : inputs) { - if (input instanceof FieldRef) { - FieldRef ref = (FieldRef) input; - int i = ref.index(); - strings.add(row.isNullAt(i) ? null : row.getString(i)); - } else { - strings.add((BinaryString) input); - } - } - return transform(strings); - } - - protected abstract BinaryString transform(List inputs); - @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/TopN.java b/paimon-api/src/main/java/org/apache/paimon/predicate/TopN.java similarity index 94% rename from paimon-common/src/main/java/org/apache/paimon/predicate/TopN.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/TopN.java index 20d34b86d2d3..afe0df002e87 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/TopN.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/TopN.java @@ -27,7 +27,6 @@ import java.util.Objects; import java.util.stream.Collectors; -import static org.apache.paimon.utils.ListUtils.isNullOrEmpty; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Represents the TopN predicate. */ @@ -39,7 +38,7 @@ public class TopN implements Serializable { private final int limit; public TopN(List orders, int limit) { - checkArgument(!isNullOrEmpty(orders), "orders should not be null or empty"); + checkArgument(orders != null && !orders.isEmpty(), "orders should not be null or empty"); this.orders = orders; this.limit = limit; } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Transform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/Transform.java similarity index 93% rename from paimon-common/src/main/java/org/apache/paimon/predicate/Transform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/Transform.java index 1b7caa1a89cc..7313df842787 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Transform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/Transform.java @@ -18,7 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalRow; import org.apache.paimon.types.DataType; import java.io.Serializable; @@ -31,7 +30,5 @@ public interface Transform extends Serializable { DataType outputType(); - Object transform(InternalRow row); - Transform copyWithNewInputs(List inputs); } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/TransformPredicate.java b/paimon-api/src/main/java/org/apache/paimon/predicate/TransformPredicate.java similarity index 68% rename from paimon-common/src/main/java/org/apache/paimon/predicate/TransformPredicate.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/TransformPredicate.java index 04549584a579..d168cb865711 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/TransformPredicate.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/TransformPredicate.java @@ -18,17 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.serializer.InternalSerializers; -import org.apache.paimon.data.serializer.ListSerializer; -import org.apache.paimon.data.serializer.NullableSerializer; -import org.apache.paimon.io.DataInputViewStreamWrapper; -import org.apache.paimon.io.DataOutputViewStreamWrapper; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -41,7 +30,7 @@ public class TransformPredicate implements Predicate { protected final Transform transform; protected final LeafFunction function; - protected transient List literals; + protected List literals; protected TransformPredicate( Transform transform, LeafFunction function, List literals) { @@ -76,18 +65,6 @@ public List fieldNames() { return names; } - @Override - public boolean test(InternalRow row) { - Object value = transform.transform(row); - return function.test(transform.outputType(), value, literals); - } - - @Override - public boolean test( - long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts) { - return true; - } - @Override public Optional negate() { return Optional.empty(); @@ -125,20 +102,4 @@ public String toString() { + literals + '}'; } - - private ListSerializer objectsSerializer() { - return new ListSerializer<>( - NullableSerializer.wrapIfNullIsNotSupported( - InternalSerializers.create(transform.outputType()))); - } - - private void writeObject(ObjectOutputStream out) throws IOException { - out.defaultWriteObject(); - objectsSerializer().serialize(literals, new DataOutputViewStreamWrapper(out)); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - literals = objectsSerializer().deserialize(new DataInputViewStreamWrapper(in)); - } } diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/UpperTransform.java b/paimon-api/src/main/java/org/apache/paimon/predicate/UpperTransform.java similarity index 82% rename from paimon-common/src/main/java/org/apache/paimon/predicate/UpperTransform.java rename to paimon-api/src/main/java/org/apache/paimon/predicate/UpperTransform.java index ffdca627c116..b6f41263c6a8 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/UpperTransform.java +++ b/paimon-api/src/main/java/org/apache/paimon/predicate/UpperTransform.java @@ -18,8 +18,6 @@ package org.apache.paimon.predicate; -import org.apache.paimon.data.BinaryString; - import java.util.List; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -34,15 +32,6 @@ public UpperTransform(List inputs) { checkArgument(inputs.size() == 1); } - @Override - public BinaryString transform(List inputs) { - BinaryString string = inputs.get(0); - if (string == null) { - return null; - } - return string.toUpperCase(); - } - @Override public Transform copyWithNewInputs(List inputs) { return new UpperTransform(inputs); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/In.java b/paimon-common/src/main/java/org/apache/paimon/predicate/In.java deleted file mode 100644 index ba96f6651195..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/In.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.predicate; - -import org.apache.paimon.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - -/** A {@link LeafFunction} to eval in. */ -public class In extends LeafFunction { - - private static final long serialVersionUID = 1L; - - public static final In INSTANCE = new In(); - - private In() {} - - @Override - public boolean test(DataType type, Object field, List literals) { - if (field == null) { - return false; - } - for (Object literal : literals) { - if (literal != null && compareLiteral(type, literal, field) == 0) { - return true; - } - } - return false; - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals) { - if (nullCount != null && rowCount == nullCount) { - return false; - } - for (Object literal : literals) { - if (literal != null - && compareLiteral(type, literal, min) >= 0 - && compareLiteral(type, literal, max) <= 0) { - return true; - } - } - return false; - } - - @Override - public Optional negate() { - return Optional.of(NotIn.INSTANCE); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitIn(fieldRef, literals); - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/Like.java b/paimon-common/src/main/java/org/apache/paimon/predicate/Like.java deleted file mode 100644 index 165352447dd4..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/Like.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.predicate; - -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.types.DataType; -import org.apache.paimon.utils.Filter; -import org.apache.paimon.utils.Pair; - -import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; -import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Caffeine; - -import org.checkerframework.checker.nullness.qual.Nullable; - -import java.util.List; -import java.util.Optional; -import java.util.regex.Pattern; - -/** A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like}. */ -public class Like extends NullFalseLeafBinaryFunction { - - public static final Like INSTANCE = new Like(); - - private static final Cache> CACHE = - Caffeine.newBuilder().softValues().executor(Runnable::run).build(); - - private Like() {} - - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - if (field == null) { - return false; - } - - BinaryString pattern = (BinaryString) patternLiteral; - Filter filter = CACHE.getIfPresent(pattern); - if (filter == null) { - filter = createFunc(type, patternLiteral); - CACHE.put(pattern, filter); - } - return filter.test((BinaryString) field); - } - - private Filter createFunc(DataType type, Object patternLiteral) { - Optional> optimized = - LikeOptimization.tryOptimize(patternLiteral); - if (optimized.isPresent()) { - NullFalseLeafBinaryFunction func = optimized.get().getKey(); - Object literal = optimized.get().getValue(); - return field -> func.test(type, field, literal); - } - // TODO optimize for chain checkers when there is no '_' - // TODO for example: "abc%def%","%abc%def","%abc%def%","abc%def" - String regex = sqlToRegexLike(patternLiteral.toString(), null); - Pattern pattern = Pattern.compile(regex); - return input -> pattern.matcher(input.toString()).matches(); - } - - private static String sqlToRegexLike(String sqlPattern, @Nullable CharSequence escapeStr) { - char escapeChar; - if (escapeStr != null) { - if (escapeStr.length() != 1) { - throw invalidEscapeCharacter(escapeStr.toString()); - } - - escapeChar = escapeStr.charAt(0); - } else { - escapeChar = '\\'; - } - - return sqlToRegexLike(sqlPattern, escapeChar); - } - - private static String sqlToRegexLike(String sqlPattern, char escapeChar) { - int len = sqlPattern.length(); - StringBuilder javaPattern = new StringBuilder(len + len); - - for (int i = 0; i < len; ++i) { - char c = sqlPattern.charAt(i); - if ("[]()|^-+*?{}$\\.".indexOf(c) >= 0) { - javaPattern.append('\\'); - } - - if (c == escapeChar) { - if (i == sqlPattern.length() - 1) { - throw invalidEscapeSequence(sqlPattern, i); - } - - char nextChar = sqlPattern.charAt(i + 1); - if (nextChar != '_' && nextChar != '%' && nextChar != escapeChar) { - throw invalidEscapeSequence(sqlPattern, i); - } - - javaPattern.append(nextChar); - ++i; - } else if (c == '_') { - javaPattern.append('.'); - } else if (c == '%') { - javaPattern.append("(?s:.*)"); - } else { - javaPattern.append(c); - } - } - - return javaPattern.toString(); - } - - private static RuntimeException invalidEscapeCharacter(String s) { - return new RuntimeException("Invalid escape character '" + s + "'"); - } - - private static RuntimeException invalidEscapeSequence(String s, int i) { - return new RuntimeException("Invalid escape sequence '" + s + "', " + i); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - Object patternLiteral) { - return true; - } - - @Override - public Optional negate() { - return Optional.empty(); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitLike(fieldRef, literals.get(0)); - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/NotIn.java b/paimon-common/src/main/java/org/apache/paimon/predicate/NotIn.java deleted file mode 100644 index d3a9715bfc56..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/NotIn.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.predicate; - -import org.apache.paimon.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static org.apache.paimon.predicate.CompareUtils.compareLiteral; - -/** A {@link LeafFunction} to eval not in. */ -public class NotIn extends LeafFunction { - - private static final long serialVersionUID = 1L; - - public static final NotIn INSTANCE = new NotIn(); - - private NotIn() {} - - @Override - public boolean test(DataType type, Object field, List literals) { - if (field == null) { - return false; - } - for (Object literal : literals) { - if (literal == null || compareLiteral(type, literal, field) == 0) { - return false; - } - } - return true; - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals) { - if (nullCount != null && rowCount == nullCount) { - return false; - } - for (Object literal : literals) { - if (literal == null - || (compareLiteral(type, literal, min) == 0 - && compareLiteral(type, literal, max) == 0)) { - return false; - } - } - return true; - } - - @Override - public Optional negate() { - return Optional.of(In.INSTANCE); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitNotIn(fieldRef, literals); - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java b/paimon-common/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java deleted file mode 100644 index 2c8b57d0d06f..000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/NullFalseLeafBinaryFunction.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.predicate; - -import org.apache.paimon.types.DataType; - -import java.util.List; - -/** Function to test a field with a literal. */ -public abstract class NullFalseLeafBinaryFunction extends LeafFunction { - - private static final long serialVersionUID = 1L; - - public abstract boolean test(DataType type, Object field, Object literal); - - public abstract boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount, Object literal); - - @Override - public boolean test(DataType type, Object field, List literals) { - if (field == null || literals.get(0) == null) { - return false; - } - return test(type, field, literals.get(0)); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals) { - if (nullCount != null) { - if (rowCount == nullCount || literals.get(0) == null) { - return false; - } - } - return test(type, rowCount, min, max, nullCount, literals.get(0)); - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateEvaluator.java b/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateEvaluator.java new file mode 100644 index 000000000000..2f7bfdca34e3 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateEvaluator.java @@ -0,0 +1,546 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.predicate; + +import org.apache.paimon.casting.CastExecutor; +import org.apache.paimon.casting.CastExecutors; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalSerializers; +import org.apache.paimon.types.DataType; +import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Caffeine; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.regex.Pattern; + +import static org.apache.paimon.predicate.CompareUtils.compareLiteral; +import static org.apache.paimon.utils.InternalRowUtils.get; + +/** Utilities to evaluate {@link Predicate} against {@link InternalRow} and stats. */ +public class PredicateEvaluator { + + private PredicateEvaluator() {} + + private static final Cache> LIKE_CACHE = + Caffeine.newBuilder().softValues().executor(Runnable::run).build(); + + public static boolean test(Predicate predicate, InternalRow row) { + if (predicate instanceof LeafPredicate) { + return testLeaf((LeafPredicate) predicate, row); + } else if (predicate instanceof TransformPredicate) { + return testTransform((TransformPredicate) predicate, row); + } else if (predicate instanceof CompoundPredicate) { + return testCompound((CompoundPredicate) predicate, row); + } + throw new UnsupportedOperationException("Unknown predicate: " + predicate.getClass()); + } + + public static boolean test( + Predicate predicate, + long rowCount, + InternalRow minValues, + InternalRow maxValues, + InternalArray nullCounts) { + if (predicate instanceof LeafPredicate) { + LeafPredicate leaf = (LeafPredicate) predicate; + Object min = get(minValues, leaf.index(), leaf.type()); + Object max = get(maxValues, leaf.index(), leaf.type()); + Long nullCount = + nullCounts.isNullAt(leaf.index()) ? null : nullCounts.getLong(leaf.index()); + if (nullCount == null || rowCount != nullCount) { + if (min == null || max == null) { + return true; + } + } + return testLeafFunction( + leaf.function(), + leaf.type(), + rowCount, + min, + max, + nullCount, + normalizeLiterals(leaf.type(), leaf.literals())); + } else if (predicate instanceof TransformPredicate) { + // Currently conservative. + return true; + } else if (predicate instanceof CompoundPredicate) { + return testCompound( + (CompoundPredicate) predicate, rowCount, minValues, maxValues, nullCounts); + } + throw new UnsupportedOperationException("Unknown predicate: " + predicate.getClass()); + } + + private static boolean testCompound(CompoundPredicate predicate, InternalRow row) { + CompoundPredicate.Function function = predicate.function(); + if (function instanceof And) { + for (Predicate child : predicate.children()) { + if (!test(child, row)) { + return false; + } + } + return true; + } else if (function instanceof Or) { + for (Predicate child : predicate.children()) { + if (test(child, row)) { + return true; + } + } + return false; + } + throw new UnsupportedOperationException( + "Unknown compound function: " + function.getClass()); + } + + private static boolean testCompound( + CompoundPredicate predicate, + long rowCount, + InternalRow minValues, + InternalRow maxValues, + InternalArray nullCounts) { + CompoundPredicate.Function function = predicate.function(); + if (function instanceof And) { + for (Predicate child : predicate.children()) { + if (!test(child, rowCount, minValues, maxValues, nullCounts)) { + return false; + } + } + return true; + } else if (function instanceof Or) { + for (Predicate child : predicate.children()) { + if (test(child, rowCount, minValues, maxValues, nullCounts)) { + return true; + } + } + return false; + } + throw new UnsupportedOperationException( + "Unknown compound function: " + function.getClass()); + } + + private static boolean testLeaf(LeafPredicate predicate, InternalRow row) { + Object value = evalTransform(predicate.transform(), row); + return testLeafFunction( + predicate.function(), + predicate.type(), + value, + normalizeLiterals(predicate.type(), predicate.literals())); + } + + private static boolean testTransform(TransformPredicate predicate, InternalRow row) { + Object value = evalTransform(predicate.transform(), row); + DataType type = predicate.transform().outputType(); + return testLeafFunction( + predicate.function, type, value, normalizeLiterals(type, predicate.literals)); + } + + private static boolean testLeafFunction( + LeafFunction function, DataType type, Object field, List literals) { + if (function instanceof LeafUnaryFunction) { + return testUnaryFunction((LeafUnaryFunction) function, field); + } + + if (function instanceof NullFalseLeafBinaryFunction) { + Object literal = firstLiteral(literals); + if (field == null || literal == null) { + return false; + } + return testNullFalseBinaryFunction( + (NullFalseLeafBinaryFunction) function, type, field, literal); + } + + if (function instanceof In) { + return testIn(type, field, literals); + } + + if (function instanceof NotIn) { + return testNotIn(type, field, literals); + } + + throw new UnsupportedOperationException("Unknown leaf function: " + function.getClass()); + } + + private static boolean testLeafFunction( + LeafFunction function, + DataType type, + long rowCount, + Object min, + Object max, + Long nullCount, + List literals) { + if (function instanceof LeafUnaryFunction) { + return testUnaryFunctionStats((LeafUnaryFunction) function, rowCount, nullCount); + } + + if (function instanceof NullFalseLeafBinaryFunction) { + Object literal = firstLiteral(literals); + if (nullCount != null) { + if (rowCount == nullCount || literal == null) { + return false; + } + } + return testNullFalseBinaryFunctionStats( + (NullFalseLeafBinaryFunction) function, type, min, max, literal); + } + + if (function instanceof In) { + return testInStats(type, rowCount, min, max, nullCount, literals); + } + + if (function instanceof NotIn) { + return testNotInStats(type, rowCount, min, max, nullCount, literals); + } + + throw new UnsupportedOperationException("Unknown leaf function: " + function.getClass()); + } + + private static boolean testUnaryFunction(LeafUnaryFunction function, Object field) { + if (function instanceof IsNull) { + return field == null; + } else if (function instanceof IsNotNull) { + return field != null; + } + throw new UnsupportedOperationException("Unknown unary function: " + function.getClass()); + } + + private static boolean testUnaryFunctionStats( + LeafUnaryFunction function, long rowCount, Long nullCount) { + if (function instanceof IsNull) { + return nullCount == null || nullCount > 0; + } else if (function instanceof IsNotNull) { + return nullCount == null || nullCount < rowCount; + } + throw new UnsupportedOperationException("Unknown unary function: " + function.getClass()); + } + + private static boolean testNullFalseBinaryFunction( + NullFalseLeafBinaryFunction function, DataType type, Object field, Object literal) { + if (function instanceof Equal) { + return compareLiteral(type, literal, field) == 0; + } else if (function instanceof NotEqual) { + return compareLiteral(type, literal, field) != 0; + } else if (function instanceof LessThan) { + return compareLiteral(type, literal, field) > 0; + } else if (function instanceof LessOrEqual) { + return compareLiteral(type, literal, field) >= 0; + } else if (function instanceof GreaterThan) { + return compareLiteral(type, literal, field) < 0; + } else if (function instanceof GreaterOrEqual) { + return compareLiteral(type, literal, field) <= 0; + } else if (function instanceof StartsWith) { + return field.toString().startsWith(literal.toString()); + } else if (function instanceof EndsWith) { + return field.toString().endsWith(literal.toString()); + } else if (function instanceof Contains) { + return field.toString().contains(literal.toString()); + } else if (function instanceof Like) { + return testLike(field, literal); + } + throw new UnsupportedOperationException("Unknown binary function: " + function.getClass()); + } + + private static boolean testNullFalseBinaryFunctionStats( + NullFalseLeafBinaryFunction function, + DataType type, + Object min, + Object max, + Object literal) { + if (function instanceof Equal) { + return compareLiteral(type, literal, min) >= 0 + && compareLiteral(type, literal, max) <= 0; + } else if (function instanceof NotEqual) { + return compareLiteral(type, literal, min) != 0 + || compareLiteral(type, literal, max) != 0; + } else if (function instanceof LessThan) { + return compareLiteral(type, literal, min) > 0; + } else if (function instanceof LessOrEqual) { + return compareLiteral(type, literal, min) >= 0; + } else if (function instanceof GreaterThan) { + return compareLiteral(type, literal, max) < 0; + } else if (function instanceof GreaterOrEqual) { + return compareLiteral(type, literal, max) <= 0; + } else if (function instanceof StartsWith) { + // Same semantics as the original StartsWith stats evaluation. + if (min == null || max == null) { + return true; + } + String minStr = min.toString(); + String maxStr = max.toString(); + String pattern = literal.toString(); + return (minStr.startsWith(pattern) || minStr.compareTo(pattern) <= 0) + && (maxStr.startsWith(pattern) || maxStr.compareTo(pattern) >= 0); + } else if (function instanceof EndsWith) { + return true; + } else if (function instanceof Contains) { + return true; + } else if (function instanceof Like) { + return true; + } + throw new UnsupportedOperationException("Unknown binary function: " + function.getClass()); + } + + private static boolean testIn(DataType type, Object field, List literals) { + if (field == null) { + return false; + } + for (Object literal : literals) { + if (literal != null && compareLiteral(type, literal, field) == 0) { + return true; + } + } + return false; + } + + private static boolean testInStats( + DataType type, + long rowCount, + Object min, + Object max, + Long nullCount, + List literals) { + if (nullCount != null && rowCount == nullCount) { + return false; + } + for (Object literal : literals) { + if (literal != null + && compareLiteral(type, literal, min) >= 0 + && compareLiteral(type, literal, max) <= 0) { + return true; + } + } + return false; + } + + private static boolean testNotIn(DataType type, Object field, List literals) { + if (field == null) { + return false; + } + for (Object literal : literals) { + if (literal == null || compareLiteral(type, literal, field) == 0) { + return false; + } + } + return true; + } + + private static boolean testNotInStats( + DataType type, + long rowCount, + Object min, + Object max, + Long nullCount, + List literals) { + if (nullCount != null && rowCount == nullCount) { + return false; + } + for (Object literal : literals) { + if (literal == null + || (compareLiteral(type, literal, min) == 0 + && compareLiteral(type, literal, max) == 0)) { + return false; + } + } + return true; + } + + private static boolean testLike(Object field, Object patternLiteral) { + String pattern = patternLiteral.toString(); + java.util.function.Predicate filter = LIKE_CACHE.getIfPresent(pattern); + if (filter == null) { + filter = createLikePredicate(patternLiteral); + LIKE_CACHE.put(pattern, filter); + } + return filter.test(field.toString()); + } + + private static java.util.function.Predicate createLikePredicate(Object patternLiteral) { + Optional> optimized = + LikeOptimization.tryOptimize(patternLiteral); + if (optimized.isPresent()) { + Object literal = optimized.get().getValue(); + NullFalseLeafBinaryFunction func = optimized.get().getKey(); + if (func instanceof Equal) { + return s -> s.equals(literal.toString()); + } else if (func instanceof StartsWith) { + return s -> s.startsWith(literal.toString()); + } else if (func instanceof EndsWith) { + return s -> s.endsWith(literal.toString()); + } else if (func instanceof Contains) { + return s -> s.contains(literal.toString()); + } + } + + String regex = sqlToRegexLike(patternLiteral.toString(), null); + Pattern compiled = Pattern.compile(regex); + return input -> compiled.matcher(input).matches(); + } + + private static String sqlToRegexLike(String sqlPattern, @Nullable CharSequence escapeStr) { + char escapeChar; + if (escapeStr != null) { + if (escapeStr.length() != 1) { + throw invalidEscapeCharacter(escapeStr.toString()); + } + + escapeChar = escapeStr.charAt(0); + } else { + escapeChar = '\\'; + } + + return sqlToRegexLike(sqlPattern, escapeChar); + } + + private static String sqlToRegexLike(String sqlPattern, char escapeChar) { + int len = sqlPattern.length(); + StringBuilder javaPattern = new StringBuilder(len + len); + + for (int i = 0; i < len; ++i) { + char c = sqlPattern.charAt(i); + if ("[]()|^-+*?{}$\\.".indexOf(c) >= 0) { + javaPattern.append('\\'); + } + + if (c == escapeChar) { + if (i == sqlPattern.length() - 1) { + throw invalidEscapeSequence(sqlPattern, i); + } + + char nextChar = sqlPattern.charAt(i + 1); + if (nextChar != '_' && nextChar != '%' && nextChar != escapeChar) { + throw invalidEscapeSequence(sqlPattern, i); + } + + javaPattern.append(nextChar); + ++i; + } else if (c == '_') { + javaPattern.append('.'); + } else if (c == '%') { + javaPattern.append("(?s:.*)"); + } else { + javaPattern.append(c); + } + } + + return javaPattern.toString(); + } + + private static RuntimeException invalidEscapeCharacter(String s) { + return new RuntimeException("Invalid escape character '" + s + "'"); + } + + private static RuntimeException invalidEscapeSequence(String s, int i) { + return new RuntimeException("Invalid escape sequence '" + s + "', " + i); + } + + private static Object firstLiteral(List literals) { + Preconditions.checkNotNull(literals, "literals should not be null"); + Preconditions.checkArgument(!literals.isEmpty(), "literals should not be empty"); + return literals.get(0); + } + + public static Object evalTransform(Transform transform, InternalRow row) { + if (transform instanceof FieldTransform) { + FieldRef fieldRef = ((FieldTransform) transform).fieldRef(); + return get(row, fieldRef.index(), fieldRef.type()); + } else if (transform instanceof CastTransform) { + CastTransform castTransform = (CastTransform) transform; + FieldRef fieldRef = (FieldRef) castTransform.inputs().get(0); + Object input = get(row, fieldRef.index(), fieldRef.type()); + @SuppressWarnings("unchecked") + CastExecutor cast = + (CastExecutor) + CastExecutors.resolve(fieldRef.type(), castTransform.outputType()); + if (cast == null) { + throw new IllegalArgumentException( + String.format( + "Cannot resolve cast executor from %s to %s", + fieldRef.type(), castTransform.outputType())); + } + return cast.cast(input); + } else if (transform instanceof UpperTransform) { + BinaryString s = evalStringInput(((UpperTransform) transform).inputs().get(0), row); + return s == null ? null : s.toUpperCase(); + } else if (transform instanceof ConcatTransform) { + List inputs = ((ConcatTransform) transform).inputs(); + List strings = new ArrayList<>(inputs.size()); + for (Object input : inputs) { + strings.add(evalStringInput(input, row)); + } + return BinaryString.concat(strings); + } else if (transform instanceof ConcatWsTransform) { + List inputs = ((ConcatWsTransform) transform).inputs(); + Preconditions.checkArgument(inputs.size() >= 2, "concat_ws needs at least 2 inputs"); + BinaryString sep = evalStringInput(inputs.get(0), row); + List parts = new ArrayList<>(); + for (int i = 1; i < inputs.size(); i++) { + BinaryString s = evalStringInput(inputs.get(i), row); + if (s != null) { + parts.add(s); + } + } + return BinaryString.concatWs(sep, parts); + } + throw new UnsupportedOperationException("Unsupported transform: " + transform.getClass()); + } + + @Nullable + private static BinaryString evalStringInput(Object input, InternalRow row) { + if (input == null) { + return null; + } + if (input instanceof FieldRef) { + FieldRef fieldRef = (FieldRef) input; + int i = fieldRef.index(); + if (row.isNullAt(i)) { + return null; + } + Object v = get(row, i, fieldRef.type()); + if (v == null) { + return null; + } + if (v instanceof BinaryString) { + return (BinaryString) v; + } + return BinaryString.fromString(v.toString()); + } + if (input instanceof BinaryString) { + return (BinaryString) input; + } + return BinaryString.fromString(input.toString()); + } + + private static List normalizeLiterals(DataType type, List literals) { + if (literals == null || literals.isEmpty()) { + return literals; + } + List converted = new ArrayList<>(literals.size()); + for (Object literal : literals) { + converted.add(InternalSerializers.create(type).copy(literal)); + } + return converted; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/RowIdPredicateVisitor.java b/paimon-common/src/main/java/org/apache/paimon/predicate/RowIdPredicateVisitor.java index 953737dd9fe9..5d2ef22d7d87 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/RowIdPredicateVisitor.java +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/RowIdPredicateVisitor.java @@ -52,8 +52,7 @@ public Optional> visit(LeafPredicate predicate) { for (Object literal : predicate.literals()) { rowIds.add((Long) literal); } - // The list output by getRangesFromList is already sorted, - // and has no overlap + // The list output by getRangesFromList is already sorted, and has no overlap return Optional.of(Range.toRanges(rowIds)); } } diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatTransformTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatTransformTest.java index e776040f89f0..a1793a6fef2a 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatTransformTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatTransformTest.java @@ -37,7 +37,7 @@ public void testConcatLiteralInputs() { inputs.add(BinaryString.fromString("ha")); inputs.add(BinaryString.fromString("-he")); ConcatTransform transform = new ConcatTransform(inputs); - Object result = transform.transform(GenericRow.of()); + Object result = PredicateEvaluator.evalTransform(transform, GenericRow.of()); assertThat(result).isEqualTo(BinaryString.fromString("ha-he")); } @@ -48,7 +48,8 @@ public void testConcatRefInputs() { inputs.add(new FieldRef(3, "f3", DataTypes.STRING())); ConcatTransform transform = new ConcatTransform(inputs); Object result = - transform.transform( + PredicateEvaluator.evalTransform( + transform, GenericRow.of( BinaryString.fromString(""), BinaryString.fromString("ha"), @@ -64,7 +65,8 @@ public void testConcatHybridInputs() { inputs.add(new FieldRef(3, "f3", DataTypes.STRING())); ConcatTransform transform = new ConcatTransform(inputs); Object result = - transform.transform( + PredicateEvaluator.evalTransform( + transform, GenericRow.of( BinaryString.fromString(""), BinaryString.fromString(""), diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatWsTransformTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatWsTransformTest.java index f85362fd4197..3599c7f4d37b 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatWsTransformTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/ConcatWsTransformTest.java @@ -38,7 +38,7 @@ public void testConcatWsLiteralInputs() { inputs.add(BinaryString.fromString("ha")); inputs.add(BinaryString.fromString("he")); ConcatWsTransform transform = new ConcatWsTransform(inputs); - Object result = transform.transform(GenericRow.of()); + Object result = PredicateEvaluator.evalTransform(transform, GenericRow.of()); assertThat(result).isEqualTo(BinaryString.fromString("ha-he")); } @@ -50,7 +50,8 @@ public void testConcatWsRefInputs() { inputs.add(new FieldRef(3, "f3", DataTypes.STRING())); ConcatWsTransform transform = new ConcatWsTransform(inputs); Object result = - transform.transform( + PredicateEvaluator.evalTransform( + transform, GenericRow.of( BinaryString.fromString(""), BinaryString.fromString("ha"), @@ -67,7 +68,8 @@ public void testConcatWsHybridInputs() { inputs.add(new FieldRef(3, "f3", DataTypes.STRING())); ConcatWsTransform transform = new ConcatWsTransform(inputs); Object result = - transform.transform( + PredicateEvaluator.evalTransform( + transform, GenericRow.of( BinaryString.fromString(""), BinaryString.fromString(""), diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateBuilderTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateBuilderTest.java index 0cc9a90b009a..48d18dfbf063 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateBuilderTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateBuilderTest.java @@ -39,11 +39,12 @@ public void testBetween() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.between(0, 1, 3); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -62,11 +63,12 @@ public void testBetweenNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.between(0, 1, null); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(false); @@ -113,9 +115,9 @@ public void testSplitAnd() { public void testIn() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.in(0, new ArrayList<>()); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); predicate = builder.in(0, Arrays.asList(1, 2)); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(10))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(10))).isEqualTo(false); } } diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateTest.java index 5e011c6e7f56..a33b973690b4 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/PredicateTest.java @@ -46,9 +46,10 @@ public void testEqual() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.equal(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -67,8 +68,9 @@ public void testEqualNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.equal(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(false); @@ -81,9 +83,10 @@ public void testNotEqual() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.notEqual(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -104,8 +107,9 @@ public void testNotEqualNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.notEqual(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(false); @@ -118,10 +122,11 @@ public void testGreater() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.greaterThan(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(6))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(6))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 4, 0L)})) .isEqualTo(false); @@ -142,8 +147,9 @@ public void testGreaterNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.greaterThan(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 4, 0L)})) .isEqualTo(false); @@ -156,10 +162,11 @@ public void testGreaterOrEqual() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.greaterOrEqual(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(6))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(6))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 4, 0L)})) .isEqualTo(false); @@ -180,8 +187,9 @@ public void testGreaterOrEqualNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.greaterOrEqual(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 4, 0L)})) .isEqualTo(false); @@ -194,10 +202,11 @@ public void testLess() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.lessThan(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(6))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(6))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(false); @@ -216,8 +225,9 @@ public void testLessNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.lessThan(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(false); @@ -230,10 +240,11 @@ public void testLessOrEqual() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.lessOrEqual(0, 5); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(5))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(6))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(6))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(false); @@ -252,8 +263,9 @@ public void testLessOrEqualNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.lessOrEqual(0, null); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(false); @@ -266,8 +278,9 @@ public void testIsNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.isNull(0); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(true); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(false); @@ -282,8 +295,9 @@ public void testIsNotNull() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); Predicate predicate = builder.isNotNull(0); - assertThat(predicate.test(GenericRow.of(4))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(6, 7, 0L)})) .isEqualTo(true); @@ -301,10 +315,11 @@ public void testIn() { Predicate predicate = builder.in(0, Arrays.asList(1, 3)); assertThat(predicate).isInstanceOf(CompoundPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -320,10 +335,11 @@ public void testInNull() { Predicate predicate = builder.in(0, Arrays.asList(1, null, 3)); assertThat(predicate).isInstanceOf(CompoundPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -339,10 +355,11 @@ public void testNotIn() { Predicate predicate = builder.notIn(0, Arrays.asList(1, 3)); assertThat(predicate).isInstanceOf(CompoundPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(1, 1, 0L)})) .isEqualTo(false); @@ -364,10 +381,11 @@ public void testNotInNull() { Predicate predicate = builder.notIn(0, Arrays.asList(1, null, 3)); assertThat(predicate).isInstanceOf(CompoundPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(1, 1, 0L)})) .isEqualTo(false); @@ -392,13 +410,15 @@ public void testEndsWith() { GenericRow max = GenericRow.of(fromString("aaba")); GenericRow min = GenericRow.of(fromString("aabb")); Integer[] nullCount = {null}; - assertThat(predicate.test(row)).isEqualTo(true); - assertThat(predicate.test(10, min, max, new GenericArray(nullCount))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, row)).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, 10, min, max, new GenericArray(nullCount))) + .isEqualTo(true); PredicateBuilder builder2 = new PredicateBuilder(RowType.of(new CharType())); Predicate predicate2 = builder2.endsWith(0, fromString("bcc")); - assertThat(predicate2.test(row)).isEqualTo(true); - assertThat(predicate2.test(10, min, max, new GenericArray(nullCount))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate2, row)).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate2, 10, min, max, new GenericArray(nullCount))) + .isEqualTo(true); } @Test @@ -413,10 +433,11 @@ public void testLargeIn() { Predicate predicate = builder.in(0, literals); assertThat(predicate).isInstanceOf(LeafPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -441,10 +462,11 @@ public void testLargeInNull() { Predicate predicate = builder.in(0, literals); assertThat(predicate).isInstanceOf(LeafPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(0, 5, 0L)})) .isEqualTo(true); @@ -468,10 +490,11 @@ public void testLargeNotIn() { Predicate predicate = builder.notIn(0, literals); assertThat(predicate).isInstanceOf(LeafPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(1, 1, 0L)})) .isEqualTo(false); @@ -502,10 +525,11 @@ public void testLargeNotInNull() { Predicate predicate = builder.notIn(0, literals); assertThat(predicate).isInstanceOf(LeafPredicate.class); - assertThat(predicate.test(GenericRow.of(1))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(2))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of((Object) null))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(1))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(2))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of((Object) null))) + .isEqualTo(false); assertThat(test(predicate, 3, new SimpleColStats[] {new SimpleColStats(1, 1, 0L)})) .isEqualTo(false); @@ -559,9 +583,16 @@ private boolean executeLike(String s, String pattern) { if (rnd.nextBoolean()) { PredicateBuilder builder = new PredicateBuilder(RowType.of(new VarCharType())); Predicate predicate = builder.like(0, fromString(pattern)); - return predicate.test(GenericRow.of(fromString(s))); + return PredicateEvaluator.test(predicate, GenericRow.of(fromString(s))); } else { - return Like.INSTANCE.test(DataTypes.STRING(), fromString(s), fromString(pattern)); + LeafPredicate predicate = + new LeafPredicate( + Like.INSTANCE, + DataTypes.STRING(), + 0, + "f0", + Arrays.asList(fromString(pattern))); + return PredicateEvaluator.test(predicate, GenericRow.of(fromString(s))); } } @@ -576,10 +607,10 @@ public void testAnd() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType(), new IntType())); Predicate predicate = PredicateBuilder.and(builder.equal(0, 3), builder.equal(1, 5)); - assertThat(predicate.test(GenericRow.of(4, 5))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3, 6))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3, 5))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(null, 5))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4, 5))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3, 6))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3, 5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(null, 5))).isEqualTo(false); assertThat( test( @@ -615,10 +646,10 @@ public void testOr() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType(), new IntType())); Predicate predicate = PredicateBuilder.or(builder.equal(0, 3), builder.equal(1, 5)); - assertThat(predicate.test(GenericRow.of(4, 6))).isEqualTo(false); - assertThat(predicate.test(GenericRow.of(3, 6))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(3, 5))).isEqualTo(true); - assertThat(predicate.test(GenericRow.of(null, 5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(4, 6))).isEqualTo(false); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3, 6))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(3, 5))).isEqualTo(true); + assertThat(PredicateEvaluator.test(predicate, GenericRow.of(null, 5))).isEqualTo(true); assertThat( test( diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/SimpleColStatsTestUtils.java b/paimon-common/src/test/java/org/apache/paimon/predicate/SimpleColStatsTestUtils.java index 988417fc4b77..b159e7bd92a2 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/SimpleColStatsTestUtils.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/SimpleColStatsTestUtils.java @@ -35,7 +35,8 @@ public static boolean test(Predicate predicate, long rowCount, SimpleColStats[] nullCounts[i] = fieldStats[i].nullCount(); } - return predicate.test( + return PredicateEvaluator.test( + predicate, rowCount, GenericRow.of(min), GenericRow.of(max), diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/TransformPredicateTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/TransformPredicateTest.java index c2f6fa32be43..0c5fc0cc3931 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/TransformPredicateTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/TransformPredicateTest.java @@ -37,7 +37,8 @@ class TransformPredicateTest { public void testReturnTrue() { TransformPredicate predicate = create(); boolean result = - predicate.test( + PredicateEvaluator.test( + predicate, GenericRow.of( BinaryString.fromString("ha"), BinaryString.fromString("-he"))); assertThat(result).isTrue(); @@ -47,7 +48,8 @@ public void testReturnTrue() { public void testReturnFalse() { TransformPredicate predicate = create(); boolean result = - predicate.test( + PredicateEvaluator.test( + predicate, GenericRow.of( BinaryString.fromString("he"), BinaryString.fromString("-he"))); assertThat(result).isFalse(); @@ -56,7 +58,7 @@ public void testReturnFalse() { @Test public void testMinMax() { TransformPredicate predicate = create(); - boolean result = predicate.test(1, null, null, null); + boolean result = PredicateEvaluator.test(predicate, 1, null, null, null); assertThat(result).isTrue(); } diff --git a/paimon-common/src/test/java/org/apache/paimon/predicate/UpperTransformTest.java b/paimon-common/src/test/java/org/apache/paimon/predicate/UpperTransformTest.java index 27436d289aa1..d1b9a9278d2a 100644 --- a/paimon-common/src/test/java/org/apache/paimon/predicate/UpperTransformTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/predicate/UpperTransformTest.java @@ -36,7 +36,7 @@ public void testNullInputs() { List inputs = new ArrayList<>(); inputs.add(null); UpperTransform transform = new UpperTransform(inputs); - Object result = transform.transform(GenericRow.of()); + Object result = PredicateEvaluator.evalTransform(transform, GenericRow.of()); assertThat(result).isNull(); } @@ -45,7 +45,7 @@ public void testNormalInputs() { List inputs = new ArrayList<>(); inputs.add(BinaryString.fromString("hello")); UpperTransform transform = new UpperTransform(inputs); - Object result = transform.transform(GenericRow.of()); + Object result = PredicateEvaluator.evalTransform(transform, GenericRow.of()); assertThat(result).isEqualTo(BinaryString.fromString("HELLO")); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java index 189270616954..61c14364097c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java @@ -23,6 +23,7 @@ import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.SimpleStatsEvolution; @@ -121,7 +122,8 @@ protected boolean filterByStats(ManifestEntry entry) { // filter by min max boolean result = - notEvolvedFilter.test( + PredicateEvaluator.test( + notEvolvedFilter, entry.file().rowCount(), stats.minValues(), stats.maxValues(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java index c2695ce6823b..83b3936cf029 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java @@ -26,6 +26,7 @@ import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.reader.DataEvolutionArray; import org.apache.paimon.reader.DataEvolutionRow; import org.apache.paimon.schema.SchemaManager; @@ -146,8 +147,12 @@ protected List postFilterManifestEntries(List entr private boolean filterByStats(List entries) { EvolutionStats stats = evolutionStats(schema, this::scanTableSchema, entries); - return inputFilter.test( - stats.rowCount(), stats.minValues(), stats.maxValues(), stats.nullCounts()); + return PredicateEvaluator.test( + inputFilter, + stats.rowCount(), + stats.minValues(), + stats.maxValues(), + stats.nullCounts()); } /** TODO: Optimize implementation of this method. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index ef6fd1e52b8e..b3ff1f37af50 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -27,6 +27,7 @@ import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; @@ -153,8 +154,12 @@ protected boolean filterByStats(ManifestEntry entry) { fieldKeyStatsConverters .getOrCreate(file.schemaId()) .evolution(file.keyStats(), file.rowCount(), null); - return notEvolvedFilter.test( - file.rowCount(), stats.minValues(), stats.maxValues(), stats.nullCounts()); + return PredicateEvaluator.test( + notEvolvedFilter, + file.rowCount(), + stats.minValues(), + stats.maxValues(), + stats.nullCounts()); } @Override @@ -282,7 +287,8 @@ private boolean filterByValueFilter(ManifestEntry entry) { fieldValueStatsConverters .getOrCreate(file.schemaId()) .evolution(file.valueStats(), file.rowCount(), file.valueStatsCols()); - return notEvolvedFilter.test( + return PredicateEvaluator.test( + notEvolvedFilter, file.rowCount(), result.minValues(), result.maxValues(), diff --git a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java index d997ad2db7aa..eb6cfb43bfdb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java +++ b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java @@ -28,6 +28,7 @@ import org.apache.paimon.format.SimpleColStats; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.statistics.FullSimpleColStatsCollector; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Pair; @@ -169,7 +170,7 @@ private DefaultPartitionPredicate(Predicate predicate) { @Override public boolean test(BinaryRow part) { - return predicate.test(part); + return PredicateEvaluator.test(predicate, part); } @Override @@ -178,7 +179,7 @@ public boolean test( InternalRow minValues, InternalRow maxValues, InternalArray nullCounts) { - return predicate.test(rowCount, minValues, maxValues, nullCounts); + return PredicateEvaluator.test(predicate, rowCount, minValues, maxValues, nullCounts); } public Predicate predicate() { @@ -274,8 +275,9 @@ public boolean test( } for (int i = 0; i < fieldNum; i++) { - if (!min[i].test(rowCount, minValues, maxValues, nullCounts) - || !max[i].test(rowCount, minValues, maxValues, nullCounts)) { + if (!PredicateEvaluator.test(min[i], rowCount, minValues, maxValues, nullCounts) + || !PredicateEvaluator.test( + max[i], rowCount, minValues, maxValues, nullCounts)) { return false; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/format/FormatTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/format/FormatTableRead.java index 5406d0c1bf79..9e356603cab8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/format/FormatTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/format/FormatTableRead.java @@ -22,6 +22,7 @@ import org.apache.paimon.disk.IOManager; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.predicate.PredicateProjectionConverter; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.table.FormatTable; @@ -104,7 +105,7 @@ private RecordReader executeFilter(RecordReader reader } Predicate finalFilter = predicate; - return reader.filter(finalFilter::test); + return reader.filter(row -> PredicateEvaluator.test(finalFilter, row)); } private RecordReader applyLimit(RecordReader reader, int limit) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java index bb4a3cce2be4..4306e7052b28 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableRead.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.variant.VariantAccessInfo; import org.apache.paimon.disk.IOManager; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.predicate.PredicateProjectionConverter; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.TableSchema; @@ -115,6 +116,6 @@ private RecordReader executeFilter(RecordReader reader } Predicate finalFilter = predicate; - return reader.filter(finalFilter::test); + return reader.filter(row -> PredicateEvaluator.test(finalFilter, row)); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java index caa8315eb9ff..2edc02e96332 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java @@ -35,6 +35,7 @@ import org.apache.paimon.predicate.LeafPredicate; import org.apache.paimon.predicate.LeafPredicateExtractor; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; @@ -264,14 +265,16 @@ private TableScan.Plan tablePlan(FileStoreTable storeTable) { scan.withBucketFilter( bucket -> { // bucket index: 1 - return bucketPredicate.test(GenericRow.of(null, bucket)); + return PredicateEvaluator.test( + bucketPredicate, GenericRow.of(null, bucket)); }); } if (levelPredicate != null) { scan.withLevelFilter( level -> { // level index: 5 - return levelPredicate.test( + return PredicateEvaluator.test( + levelPredicate, GenericRow.of(null, null, null, null, null, level)); }); } else { diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index 75a8271ae16d..8742e8e414d1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -40,6 +40,7 @@ import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction; import org.apache.paimon.operation.commit.RetryCommitResult; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.SchemaUtils; @@ -637,7 +638,7 @@ public void testDropPartitions() throws Exception { List expectedKvs = new ArrayList<>(); for (Map.Entry> entry : data.entrySet()) { - if (partitionFilter.test(entry.getKey())) { + if (PredicateEvaluator.test(partitionFilter, entry.getKey())) { continue; } expectedKvs.addAll(entry.getValue()); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/ChainTableUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/ChainTableUtilsTest.java index 99ba22c451a9..9a9a450bb48f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/ChainTableUtilsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/ChainTableUtilsTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -120,7 +121,7 @@ public void testCreateTriangularPredicate() { partitionConverter, (Integer i, Object j) -> builder.equal(i, j), (Integer i, Object j) -> builder.lessThan(i, j)); - Assertions.assertTrue(!predicate.test(partitionValue)); + Assertions.assertTrue(!PredicateEvaluator.test(predicate, partitionValue)); List predicates = new ArrayList<>(); predicates.add(builder.lessThan(0, partitionValue.getString(0))); List subPredicates = new ArrayList<>(); @@ -150,7 +151,7 @@ public void testCreateLinearPredicate() { partitionValue, partitionConverter, (Integer i, Object j) -> builder.equal(i, j)); - Assertions.assertTrue(predicate.test(partitionValue)); + Assertions.assertTrue(PredicateEvaluator.test(predicate, partitionValue)); List predicates = new ArrayList<>(); predicates.add(builder.equal(0, partitionValue.getString(0))); predicates.add(builder.equal(1, partitionValue.getString(1))); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java index cda824cc946a..12277a994ddb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java @@ -32,6 +32,7 @@ import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.sort.BinaryExternalSortBuffer; import org.apache.paimon.table.FileStoreTable; @@ -197,7 +198,7 @@ protected void bootstrap() throws Exception { new RecordReaderIterator<>(reader.toRecordReader(reader.nextSplits(), true))) { while (batch.hasNext()) { InternalRow row = batch.next(); - if (predicate == null || predicate.test(row)) { + if (predicate == null || PredicateEvaluator.test(predicate, row)) { bulkLoadSorter.write(GenericRow.of(toKeyBytes(row), toValueBytes(row))); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java index ac0f00d89594..cf352f5c98a3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java @@ -25,6 +25,7 @@ import org.apache.paimon.mergetree.compact.ConcatRecordReader; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.reader.ReaderSupplier; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.table.source.DataSplit; @@ -140,7 +141,7 @@ public RecordReader toRecordReader(List splits, boolean useP } if (projectedPredicate != null) { - reader = reader.filter(projectedPredicate::test); + reader = reader.filter(row -> PredicateEvaluator.test(projectedPredicate, row)); } if (cacheRowFilter != null) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java index 546bc5a60a00..67d32ea40f9d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/NoPrimaryKeyLookupTable.java @@ -24,6 +24,7 @@ import org.apache.paimon.lookup.ListState; import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.types.RowKind; import org.apache.paimon.utils.KeyProjectedRow; import org.apache.paimon.utils.TypeUtils; @@ -82,7 +83,7 @@ public void refresh(Iterator incremental) throws IOException { protected void refreshRow(InternalRow row, Predicate predicate) throws IOException { joinKeyRow.replaceRow(row); if (row.getRowKind() == RowKind.INSERT || row.getRowKind() == RowKind.UPDATE_AFTER) { - if (predicate == null || predicate.test(row)) { + if (predicate == null || PredicateEvaluator.test(predicate, row)) { state.add(joinKeyRow, row); } } else { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java index 485a9cfaa6c1..8c27f7b81243 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyLookupTable.java @@ -24,6 +24,7 @@ import org.apache.paimon.lookup.ValueState; import org.apache.paimon.lookup.rocksdb.RocksDBBulkLoader; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.RowKind; import org.apache.paimon.utils.KeyProjectedRow; @@ -105,7 +106,7 @@ protected void refreshRow(InternalRow row, Predicate predicate) throws IOExcepti } if (row.getRowKind() == RowKind.INSERT || row.getRowKind() == RowKind.UPDATE_AFTER) { - if (predicate == null || predicate.test(row)) { + if (predicate == null || PredicateEvaluator.test(predicate, row)) { tableState.put(primaryKeyRow, row); } else { // The new record under primary key is filtered diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java index 668570d77d12..715e4834eb0e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/SecondaryIndexLookupTable.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.serializer.InternalSerializers; import org.apache.paimon.lookup.SetState; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.types.RowKind; import org.apache.paimon.utils.KeyProjectedRow; import org.apache.paimon.utils.TypeUtils; @@ -94,7 +95,7 @@ protected void refreshRow(InternalRow row, Predicate predicate) throws IOExcepti indexState.retract(secKeyRow.replaceRow(previous), primaryKeyRow); } - if (predicate == null || predicate.test(row)) { + if (predicate == null || PredicateEvaluator.test(predicate, row)) { tableState.put(primaryKeyRow, row); indexState.add(secKeyRow.replaceRow(row), primaryKeyRow); } else { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateConverterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateConverterTest.java index c304a704344b..41432fa4ae57 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateConverterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateConverterTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.format.SimpleColStats; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.predicate.SimpleColStatsTestUtils; import org.apache.flink.table.api.DataTypes; @@ -285,7 +286,10 @@ public void testStartsWith( IntStream.range(0, valuesList.size()) .forEach( i -> - assertThat(predicate.test(GenericRow.of(valuesList.get(i)))) + assertThat( + PredicateEvaluator.test( + predicate, + GenericRow.of(valuesList.get(i)))) .isEqualTo(expectedForValues.get(i))); IntStream.range(0, rowCountList.size()) .forEach( diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java index 4248d07d769f..a470b57260ce 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.Timestamp; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; +import org.apache.paimon.predicate.PredicateEvaluator; import org.apache.paimon.types.CharType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DateType; @@ -164,18 +165,19 @@ public void testAll() { GenericRow row = GenericRow.of(fromString("aabc")); GenericRow max = GenericRow.of(fromString("xasxwsa")); GenericRow min = GenericRow.of(fromString("aaaaa")); - boolean test = endsWithPre.test(row); + boolean test = PredicateEvaluator.test(endsWithPre, row); Integer[] nullCount = {null}; - boolean test1 = endsWithPre.test(10, min, max, new GenericArray(nullCount)); + boolean test1 = + PredicateEvaluator.test(endsWithPre, 10, min, max, new GenericArray(nullCount)); assertThat(test).isEqualTo(true); assertThat(test1).isEqualTo(true); // StringContains StringContains stringContains = StringContains.apply("id", "aa"); Predicate contains = converter01.convert(stringContains); - assertThat(contains.test(row)).isEqualTo(true); - assertThat(contains.test(max)).isEqualTo(false); - assertThat(contains.test(min)).isEqualTo(true); + assertThat(PredicateEvaluator.test(contains, row)).isEqualTo(true); + assertThat(PredicateEvaluator.test(contains, max)).isEqualTo(false); + assertThat(PredicateEvaluator.test(contains, min)).isEqualTo(true); } @Test @@ -206,7 +208,7 @@ public void testChar() { StringEndsWith endsWith = StringEndsWith.apply("id", "abc"); Predicate endsWithPre = converter.convert(endsWith); GenericRow row = GenericRow.of(fromString("aabc")); - boolean test = endsWithPre.test(row); + boolean test = PredicateEvaluator.test(endsWithPre, row); assertThat(test).isEqualTo(true); }