diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java index 0726c061e05e..e6aab8da5b04 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/FileIndexReader.java @@ -46,6 +46,11 @@ public FileIndexResult visitStartsWith(FieldRef fieldRef, Object literal) { return REMAIN; } + @Override + public FileIndexResult visitEndsWith(FieldRef fieldRef, Object literal) { + return REMAIN; + } + @Override public FileIndexResult visitLessThan(FieldRef fieldRef, Object literal) { return REMAIN; diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java index 77e66f467598..b417b9764e37 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/empty/EmptyFileIndexReader.java @@ -47,6 +47,11 @@ public FileIndexResult visitStartsWith(FieldRef fieldRef, Object literal) { return SKIP; } + @Override + public FileIndexResult visitEndsWith(FieldRef fieldRef, Object literal) { + return SKIP; + } + @Override public FileIndexResult visitLessThan(FieldRef fieldRef, Object literal) { return SKIP; diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java b/paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java new file mode 100644 index 000000000000..13fcf1249038 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/EndsWith.java @@ -0,0 +1,63 @@ +/* + * 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 java.util.List; +import java.util.Optional; + +/** + * A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like '%abc' or filter like + * '_abc'}. + */ +public class EndsWith extends NullFalseLeafBinaryFunction { + + public static final EndsWith INSTANCE = new EndsWith(); + + 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(); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitEndsWith(fieldRef, literals.get(0)); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java b/paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java index f15e85d7f515..c29b001f01ed 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/FunctionVisitor.java @@ -50,6 +50,8 @@ default T visit(CompoundPredicate predicate) { T visitStartsWith(FieldRef fieldRef, Object literal); + T visitEndsWith(FieldRef fieldRef, Object literal); + T visitLessThan(FieldRef fieldRef, Object literal); T visitGreaterOrEqual(FieldRef fieldRef, Object literal); diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java b/paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java index f085dcb53d3d..83670c99abe5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/OnlyPartitionKeyEqualVisitor.java @@ -57,6 +57,11 @@ public Boolean visitStartsWith(FieldRef fieldRef, Object literal) { return false; } + @Override + public Boolean visitEndsWith(FieldRef fieldRef, Object literal) { + return false; + } + @Override public Boolean visitLessThan(FieldRef fieldRef, Object literal) { return false; diff --git a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateBuilder.java b/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateBuilder.java index 385775736c2c..cf85cc3b4dfe 100644 --- a/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateBuilder.java +++ b/paimon-common/src/main/java/org/apache/paimon/predicate/PredicateBuilder.java @@ -107,6 +107,10 @@ public Predicate startsWith(int idx, Object patternLiteral) { return leaf(StartsWith.INSTANCE, idx, patternLiteral); } + public Predicate endsWith(int idx, Object patternLiteral) { + return leaf(EndsWith.INSTANCE, idx, patternLiteral); + } + public Predicate leaf(NullFalseLeafBinaryFunction function, int idx, Object literal) { DataField field = rowType.getFields().get(idx); return new LeafPredicate(function, field.type(), idx, field.name(), singletonList(literal)); 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 dc311bb5fc06..fe9f02904462 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 @@ -18,10 +18,12 @@ package org.apache.paimon.predicate; +import org.apache.paimon.data.GenericArray; import org.apache.paimon.data.GenericRow; import org.apache.paimon.format.SimpleColStats; import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; import org.junit.jupiter.api.Test; @@ -29,6 +31,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.paimon.data.BinaryString.fromString; import static org.apache.paimon.predicate.SimpleColStatsTestUtils.test; import static org.assertj.core.api.Assertions.assertThat; @@ -377,6 +380,19 @@ public void testNotInNull() { .isEqualTo(false); } + @Test + public void testEndsWith() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new VarCharType())); + Predicate predicate = builder.endsWith(0, fromString("bcc")); + GenericRow row = GenericRow.of(fromString("aabbcc")); + + 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); + } + @Test public void testLargeIn() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcPredicateFunctionVisitor.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcPredicateFunctionVisitor.java index aeadf1aa0664..81226dad50a8 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcPredicateFunctionVisitor.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcPredicateFunctionVisitor.java @@ -64,6 +64,11 @@ public Optional visitStartsWith(FieldRef fieldRef, Object return Optional.empty(); } + @Override + public Optional visitEndsWith(FieldRef fieldRef, Object literal) { + return Optional.empty(); + } + @Override public Optional visitLessThan(FieldRef fieldRef, Object literal) { return convertBinary(fieldRef, literal, OrcFilters.LessThan::new); diff --git a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java index ef36cc6f892a..27850ae9a790 100644 --- a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java +++ b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java @@ -141,6 +141,11 @@ public FilterPredicate visitStartsWith(FieldRef fieldRef, Object literal) { throw new UnsupportedOperationException(); } + @Override + public FilterPredicate visitEndsWith(FieldRef fieldRef, Object literal) { + throw new UnsupportedOperationException(); + } + @Override public FilterPredicate visitIn(FieldRef fieldRef, List literals) { throw new UnsupportedOperationException(); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkFilterConverter.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkFilterConverter.java index b600a0275268..7d049948981f 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkFilterConverter.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkFilterConverter.java @@ -36,6 +36,7 @@ import org.apache.spark.sql.sources.LessThanOrEqual; import org.apache.spark.sql.sources.Not; import org.apache.spark.sql.sources.Or; +import org.apache.spark.sql.sources.StringEndsWith; import org.apache.spark.sql.sources.StringStartsWith; import java.util.Arrays; @@ -61,7 +62,8 @@ public class SparkFilterConverter { "And", "Or", "Not", - "StringStartsWith"); + "StringStartsWith", + "StringEndsWith"); private final RowType rowType; private final PredicateBuilder builder; @@ -141,6 +143,11 @@ public Predicate convert(Filter filter) { int index = fieldIndex(startsWith.attribute()); Object literal = convertLiteral(index, startsWith.value()); return builder.startsWith(index, literal); + } else if (filter instanceof StringEndsWith) { + StringEndsWith endsWith = (StringEndsWith) filter; + int index = fieldIndex(endsWith.attribute()); + Object literal = convertLiteral(index, endsWith.value()); + return builder.endsWith(index, literal); } // TODO: AlwaysTrue, AlwaysFalse diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java index 3778d4575115..79725a181390 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java @@ -18,6 +18,8 @@ package org.apache.paimon.spark; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.Timestamp; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; @@ -38,6 +40,7 @@ import org.apache.spark.sql.sources.LessThan; import org.apache.spark.sql.sources.LessThanOrEqual; import org.apache.spark.sql.sources.Not; +import org.apache.spark.sql.sources.StringEndsWith; import org.apache.spark.sql.sources.StringStartsWith; import org.junit.jupiter.api.Test; @@ -51,6 +54,7 @@ import java.util.Collections; import java.util.List; +import static org.apache.paimon.data.BinaryString.fromString; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.catchThrowableOfType; @@ -149,6 +153,20 @@ public void testAll() { Predicate expectedLargeIn = builder.in(0, Arrays.asList(literals)); Predicate actualLargeIn = converter.convert(largeIn); assertThat(actualLargeIn).isEqualTo(expectedLargeIn); + + RowType rowType01 = + new RowType(Collections.singletonList(new DataField(0, "id", new VarCharType()))); + SparkFilterConverter converter01 = new SparkFilterConverter(rowType01); + StringEndsWith endsWith = StringEndsWith.apply("id", "abc"); + Predicate endsWithPre = converter01.convert(endsWith); + GenericRow row = GenericRow.of(fromString("aabc")); + GenericRow max = GenericRow.of(fromString("xasxwsa")); + GenericRow min = GenericRow.of(fromString("aaaaa")); + boolean test = endsWithPre.test(row); + Integer[] nullCount = {null}; + boolean test1 = endsWithPre.test(10, min, max, new GenericArray(nullCount)); + assertThat(test).isEqualTo(true); + assertThat(test1).isEqualTo(true); } @Test