From 6645a8b8d2a5f6b708d6131f771020b4e630e51d Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:41:53 +0800 Subject: [PATCH 01/22] [flink]fix and add log --- .../java/com/alibaba/fluss/predicate/And.java | 68 +++ .../alibaba/fluss/predicate/CompareUtils.java | 59 +++ .../fluss/predicate/CompoundPredicate.java | 115 +++++ .../com/alibaba/fluss/predicate/Contains.java | 64 +++ .../com/alibaba/fluss/predicate/EndsWith.java | 67 +++ .../com/alibaba/fluss/predicate/Equal.java | 59 +++ .../com/alibaba/fluss/predicate/FieldRef.java | 80 ++++ .../fluss/predicate/FunctionVisitor.java | 84 ++++ .../fluss/predicate/GreaterOrEqual.java | 59 +++ .../alibaba/fluss/predicate/GreaterThan.java | 59 +++ .../java/com/alibaba/fluss/predicate/In.java | 84 ++++ .../fluss/predicate/InPredicateVisitor.java | 57 +++ .../alibaba/fluss/predicate/IsNotNull.java | 56 +++ .../com/alibaba/fluss/predicate/IsNull.java | 56 +++ .../alibaba/fluss/predicate/LeafFunction.java | 66 +++ .../fluss/predicate/LeafPredicate.java | 180 +++++++ .../predicate/LeafPredicateExtractor.java | 48 ++ .../fluss/predicate/LeafUnaryFunction.java | 54 +++ .../alibaba/fluss/predicate/LessOrEqual.java | 59 +++ .../com/alibaba/fluss/predicate/LessThan.java | 59 +++ .../com/alibaba/fluss/predicate/NotEqual.java | 59 +++ .../com/alibaba/fluss/predicate/NotIn.java | 84 ++++ .../NullFalseLeafBinaryFunction.java | 62 +++ .../OnlyPartitionKeyEqualVisitor.java | 124 +++++ .../java/com/alibaba/fluss/predicate/Or.java | 68 +++ .../predicate/PartitionPredicateVisitor.java | 52 ++ .../alibaba/fluss/predicate/Predicate.java | 49 ++ .../fluss/predicate/PredicateBuilder.java | 446 ++++++++++++++++++ .../PredicateProjectionConverter.java | 70 +++ .../predicate/PredicateReplaceVisitor.java | 45 ++ .../fluss/predicate/PredicateVisitor.java | 31 ++ .../alibaba/fluss/predicate/StartsWith.java | 71 +++ .../java/org/apache/fluss/types/RowType.java | 10 + .../apache/fluss/utils/BinaryStringUtils.java | 14 +- .../org/apache/fluss/utils/DateTimeUtils.java | 4 +- .../fluss/predicate/PredicateBuilderTest.java | 86 ++++ .../fluss/predicate/PredicateTest.java | 408 ++++++++++++++++ .../fluss/flink/source/FlinkTableSource.java | 138 +++--- .../enumerator/FlinkSourceEnumerator.java | 145 +++--- .../flink/source/FlinkTableSourceITCase.java | 33 ++ .../enumerator/FlinkSourceEnumeratorTest.java | 15 +- 41 files changed, 3274 insertions(+), 173 deletions(-) create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java create mode 100644 fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java create mode 100644 fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java new file mode 100644 index 0000000000..d8b74b1a37 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java @@ -0,0 +1,68 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.InternalRow; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link CompoundPredicate.Function} to eval and. */ +public class And extends CompoundPredicate.Function { + + private static final long serialVersionUID = 1L; + + public static final And INSTANCE = new And(); + + private And() {} + + @Override + public boolean test(InternalRow row, List children) { + for (Predicate child : children) { + if (!child.test(row)) { + return false; + } + } + return true; + } + + @Override + public Optional negate(List children) { + List negatedChildren = new ArrayList<>(); + for (Predicate child : children) { + Optional negatedChild = child.negate(); + if (negatedChild.isPresent()) { + negatedChildren.add(negatedChild.get()); + } else { + return Optional.empty(); + } + } + return Optional.of(new CompoundPredicate(Or.INSTANCE, negatedChildren)); + } + + @Override + public T visit(FunctionVisitor visitor, List children) { + return visitor.visitAnd(children); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java new file mode 100644 index 0000000000..5059e90f35 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.types.DataType; + +import static java.lang.Math.min; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Utils for comparator. */ +public class CompareUtils { + private CompareUtils() {} + + public static int compareLiteral(DataType type, Object v1, Object v2) { + if (v1 instanceof Comparable) { + if (v1 instanceof BinaryString) { + v1 = ((BinaryString) v1).toString(); + } + if (v2 instanceof BinaryString) { + v2 = ((BinaryString) v2).toString(); + } + return ((Comparable) v1).compareTo(v2); + } else if (v1 instanceof byte[]) { + return compare((byte[]) v1, (byte[]) v2); + } else { + throw new RuntimeException("Unsupported type: " + type); + } + } + + private static int compare(byte[] first, byte[] second) { + for (int x = 0; x < min(first.length, second.length); x++) { + int cmp = first[x] - second[x]; + if (cmp != 0) { + return cmp; + } + } + return first.length - second.length; + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java new file mode 100644 index 0000000000..f87dd1edd6 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java @@ -0,0 +1,115 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.InternalRow; + +import java.io.Serializable; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * Non-leaf node in a {@link Predicate} tree. Its evaluation result depends on the results of its + * children. + */ +public class CompoundPredicate implements Predicate { + + private final Function function; + private final List children; + + public CompoundPredicate(Function function, List children) { + this.function = function; + this.children = children; + } + + public Function function() { + return function; + } + + public List children() { + return children; + } + + @Override + public boolean test(InternalRow row) { + return function.test(row, children); + } + + @Override + public Optional negate() { + return function.negate(children); + } + + @Override + public T visit(PredicateVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof CompoundPredicate)) { + return false; + } + CompoundPredicate that = (CompoundPredicate) o; + return Objects.equals(function, that.function) && Objects.equals(children, that.children); + } + + @Override + public int hashCode() { + return Objects.hash(function, children); + } + + @Override + public String toString() { + return function + "(" + children + ")"; + } + + /** 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 Optional negate(List children); + + public abstract T visit(FunctionVisitor visitor, List children); + + @Override + public int hashCode() { + return this.getClass().getName().hashCode(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + return o != null && getClass() == o.getClass(); + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java new file mode 100644 index 0000000000..eab425a927 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java @@ -0,0 +1,64 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like '%abc%'}. */ +public class Contains extends NullFalseLeafBinaryFunction { + + public static final Contains INSTANCE = new Contains(); + + 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(); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitContains(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java new file mode 100644 index 0000000000..ba397ca6ca --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java @@ -0,0 +1,67 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java new file mode 100644 index 0000000000..3105aab1ac --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval equal. */ +public class Equal extends NullFalseLeafBinaryFunction { + + public static final Equal INSTANCE = new Equal(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitEqual(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java new file mode 100644 index 0000000000..732b188e37 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java @@ -0,0 +1,80 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.io.Serializable; +import java.util.Objects; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A reference to a field in an input. */ +public class FieldRef implements Serializable { + + private static final long serialVersionUID = 1L; + + private final int index; + private final String name; + private final DataType type; + + public FieldRef(int index, String name, DataType type) { + this.index = index; + this.name = name; + this.type = type; + } + + public int index() { + return index; + } + + public String name() { + return name; + } + + public DataType type() { + return type; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FieldRef fieldRef = (FieldRef) o; + return index == fieldRef.index + && Objects.equals(name, fieldRef.name) + && Objects.equals(type, fieldRef.type); + } + + @Override + public int hashCode() { + return Objects.hash(index, name, type); + } + + @Override + public String toString() { + return "FieldRef{" + "index=" + index + ", name='" + name + '\'' + ", type=" + type + '}'; + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java new file mode 100644 index 0000000000..7c2133be78 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java @@ -0,0 +1,84 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.List; +import java.util.stream.Collectors; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link PredicateVisitor} to visit functions. */ +public interface FunctionVisitor extends PredicateVisitor { + + @Override + default T visit(LeafPredicate predicate) { + return predicate.function().visit(this, predicate.fieldRef(), predicate.literals()); + } + + @Override + default T visit(CompoundPredicate predicate) { + return predicate + .function() + .visit( + this, + predicate.children().stream() + .map(p -> p.visit(this)) + .collect(Collectors.toList())); + } + + // ----------------- Unary functions ------------------------ + + T visitIsNotNull(FieldRef fieldRef); + + T visitIsNull(FieldRef fieldRef); + + // ----------------- Binary functions ------------------------ + + T visitStartsWith(FieldRef fieldRef, Object literal); + + T visitEndsWith(FieldRef fieldRef, Object literal); + + T visitContains(FieldRef fieldRef, Object literal); + + T visitLessThan(FieldRef fieldRef, Object literal); + + T visitGreaterOrEqual(FieldRef fieldRef, Object literal); + + T visitNotEqual(FieldRef fieldRef, Object literal); + + T visitLessOrEqual(FieldRef fieldRef, Object literal); + + T visitEqual(FieldRef fieldRef, Object literal); + + T visitGreaterThan(FieldRef fieldRef, Object literal); + + // ----------------- Other functions ------------------------ + + T visitIn(FieldRef fieldRef, List literals); + + T visitNotIn(FieldRef fieldRef, List literals); + + // ----------------- Compound functions ------------------------ + + T visitAnd(List children); + + T visitOr(List children); +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java new file mode 100644 index 0000000000..9198e24dee --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval greater or equal. */ +public class GreaterOrEqual extends NullFalseLeafBinaryFunction { + + public static final GreaterOrEqual INSTANCE = new GreaterOrEqual(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitGreaterOrEqual(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java new file mode 100644 index 0000000000..e70ffe50a5 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link LeafFunction} to eval greater. */ +public class GreaterThan extends NullFalseLeafBinaryFunction { + + public static final GreaterThan INSTANCE = new GreaterThan(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitGreaterThan(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java new file mode 100644 index 0000000000..24b1244ddb --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java @@ -0,0 +1,84 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java new file mode 100644 index 0000000000..5cd87ebc1a --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java @@ -0,0 +1,57 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A utils to handle {@link Predicate}. */ +public class InPredicateVisitor { + + /** + * Method for handling with In CompoundPredicate. + * + * @param predicate CompoundPredicate to traverse handle + * @param leafName LeafPredicate name + */ + public static Optional> extractInElements(Predicate predicate, String leafName) { + if (!(predicate instanceof CompoundPredicate)) { + return Optional.empty(); + } + + CompoundPredicate compoundPredicate = (CompoundPredicate) predicate; + List leafValues = new ArrayList<>(); + List children = compoundPredicate.children(); + for (Predicate leaf : children) { + if (leaf instanceof LeafPredicate + && (((LeafPredicate) leaf).function() instanceof Equal) + && leaf.visit(LeafPredicateExtractor.INSTANCE).get(leafName) != null) { + leafValues.add(((LeafPredicate) leaf).literals().get(0)); + } else { + return Optional.empty(); + } + } + return Optional.of(leafValues); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java new file mode 100644 index 0000000000..9defa44fd3 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java @@ -0,0 +1,56 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval is not null. */ +public class IsNotNull extends LeafUnaryFunction { + + public static final IsNotNull INSTANCE = new IsNotNull(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitIsNotNull(fieldRef); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java new file mode 100644 index 0000000000..303d282b0a --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java @@ -0,0 +1,56 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval is null. */ +public class IsNull extends LeafUnaryFunction { + + public static final IsNull INSTANCE = new IsNull(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitIsNull(fieldRef); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java new file mode 100644 index 0000000000..5a2f7b5f13 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java @@ -0,0 +1,66 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** 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 + public int hashCode() { + return this.getClass().getName().hashCode(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + return o != null && getClass() == o.getClass(); + } + + public abstract T visit( + FunctionVisitor visitor, FieldRef fieldRef, List literals); + + @Override + public String toString() { + return getClass().getSimpleName(); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java new file mode 100644 index 0000000000..ba9783ffa5 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java @@ -0,0 +1,180 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.InternalRow; +import com.alibaba.fluss.types.DataType; +import com.alibaba.fluss.types.DecimalType; +import com.alibaba.fluss.types.LocalZonedTimestampType; +import com.alibaba.fluss.types.TimestampType; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Leaf node of a {@link Predicate} tree. Compares a field in the row with literals. */ +public class LeafPredicate implements Predicate { + + private static final long serialVersionUID = 1L; + + private final LeafFunction function; + private final DataType type; + private final int fieldIndex; + private final String fieldName; + + private List literals; + + public LeafPredicate( + LeafFunction function, + DataType type, + int fieldIndex, + String fieldName, + List literals) { + this.function = function; + this.type = type; + this.fieldIndex = fieldIndex; + this.fieldName = fieldName; + this.literals = literals; + } + + public LeafFunction function() { + return function; + } + + public DataType type() { + return type; + } + + public int index() { + return fieldIndex; + } + + public String fieldName() { + return fieldName; + } + + public FieldRef fieldRef() { + return new FieldRef(fieldIndex, fieldName, type); + } + + public List literals() { + return literals; + } + + public LeafPredicate copyWithNewIndex(int fieldIndex) { + return new LeafPredicate(function, type, fieldIndex, fieldName, literals); + } + + @Override + public boolean test(InternalRow row) { + return function.test(type, get(row, fieldIndex, type), literals); + } + + @Override + public Optional negate() { + return function.negate() + .map(negate -> new LeafPredicate(negate, type, fieldIndex, fieldName, literals)); + } + + @Override + public T visit(PredicateVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LeafPredicate that = (LeafPredicate) o; + return fieldIndex == that.fieldIndex + && Objects.equals(fieldName, that.fieldName) + && Objects.equals(function, that.function) + && Objects.equals(type, that.type) + && Objects.equals(literals, that.literals); + } + + @Override + public int hashCode() { + return Objects.hash(function, type, fieldIndex, fieldName, literals); + } + + @Override + public String toString() { + String literalsStr; + if (literals == null || literals.isEmpty()) { + literalsStr = ""; + } else if (literals.size() == 1) { + literalsStr = Objects.toString(literals.get(0)); + } else { + literalsStr = literals.toString(); + } + return literalsStr.isEmpty() + ? function + "(" + fieldName + ")" + : function + "(" + fieldName + ", " + literalsStr + ")"; + } + + public static Object get(InternalRow internalRow, int pos, DataType fieldType) { + if (internalRow.isNullAt(pos)) { + return null; + } + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + return internalRow.getBoolean(pos); + case TINYINT: + return internalRow.getByte(pos); + case SMALLINT: + return internalRow.getShort(pos); + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + return internalRow.getInt(pos); + case BIGINT: + return internalRow.getLong(pos); + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) fieldType; + return internalRow.getTimestampNtz(pos, timestampType.getPrecision()); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType lzTs = (LocalZonedTimestampType) fieldType; + return internalRow.getTimestampNtz(pos, lzTs.getPrecision()); + case FLOAT: + return internalRow.getFloat(pos); + case DOUBLE: + return internalRow.getDouble(pos); + case CHAR: + case STRING: + return internalRow.getString(pos); + case DECIMAL: + DecimalType decimalType = (DecimalType) fieldType; + return internalRow.getDecimal( + pos, decimalType.getPrecision(), decimalType.getScale()); + case BINARY: + return internalRow.getBytes(pos); + default: + throw new UnsupportedOperationException("Unsupported type: " + fieldType); + } + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java new file mode 100644 index 0000000000..0d6375e3f1 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java @@ -0,0 +1,48 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Extract leaf predicate for field names. */ +public class LeafPredicateExtractor implements PredicateVisitor> { + + public static final LeafPredicateExtractor INSTANCE = new LeafPredicateExtractor(); + + @Override + public Map visit(LeafPredicate predicate) { + return Collections.singletonMap(predicate.fieldName(), predicate); + } + + @Override + public Map visit(CompoundPredicate predicate) { + if (predicate.function() instanceof And) { + Map leafPredicates = new HashMap<>(); + predicate.children().stream().map(p -> p.visit(this)).forEach(leafPredicates::putAll); + return leafPredicates; + } + return Collections.emptyMap(); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java new file mode 100644 index 0000000000..a1ce6721c1 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java @@ -0,0 +1,54 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Function to test a field. */ +public abstract class LeafUnaryFunction extends LeafFunction { + + private static final long serialVersionUID = 1L; + + public abstract boolean test(DataType type, Object value); + + public abstract boolean test( + DataType type, long rowCount, Object min, Object max, Long nullCount); + + @Override + public boolean test(DataType type, Object value, List literals) { + return test(type, value); + } + + @Override + public boolean test( + DataType type, + long rowCount, + Object min, + Object max, + Long nullCount, + List literals) { + return test(type, rowCount, min, max, nullCount); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java new file mode 100644 index 0000000000..f4b6b382c6 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ +public class LessOrEqual extends NullFalseLeafBinaryFunction { + + public static final LessOrEqual INSTANCE = new LessOrEqual(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitLessOrEqual(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java new file mode 100644 index 0000000000..40554da7f6 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ +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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitLessThan(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java new file mode 100644 index 0000000000..4f8ec5bdc7 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java @@ -0,0 +1,59 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link NullFalseLeafBinaryFunction} to eval not equal. */ +public class NotEqual extends NullFalseLeafBinaryFunction { + + public static final NotEqual INSTANCE = new NotEqual(); + + 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); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitNotEqual(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java new file mode 100644 index 0000000000..f20d6c5839 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java @@ -0,0 +1,84 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java new file mode 100644 index 0000000000..ffeaa896db --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java @@ -0,0 +1,62 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.types.DataType; + +import java.util.List; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java new file mode 100644 index 0000000000..80913446f0 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java @@ -0,0 +1,124 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Visit the predicate and check if it only contains partition keys and can be push down. */ +public class OnlyPartitionKeyEqualVisitor implements FunctionVisitor { + + private final List partitionKeys; + + private final Map partitions; + + public OnlyPartitionKeyEqualVisitor(List partitionKeys) { + this.partitionKeys = partitionKeys; + partitions = new HashMap<>(); + } + + public Map partitions() { + return partitions; + } + + @Override + public Boolean visitIsNotNull(FieldRef fieldRef) { + return false; + } + + @Override + public Boolean visitIsNull(FieldRef fieldRef) { + return false; + } + + @Override + public Boolean visitStartsWith(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitEndsWith(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitContains(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitLessThan(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitGreaterOrEqual(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitNotEqual(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitLessOrEqual(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitEqual(FieldRef fieldRef, Object literal) { + boolean contains = partitionKeys.contains(fieldRef.name()); + if (contains) { + partitions.put(fieldRef.name(), literal.toString()); + return true; + } + return false; + } + + @Override + public Boolean visitGreaterThan(FieldRef fieldRef, Object literal) { + return false; + } + + @Override + public Boolean visitIn(FieldRef fieldRef, List literals) { + return false; + } + + @Override + public Boolean visitNotIn(FieldRef fieldRef, List literals) { + return false; + } + + @Override + public Boolean visitAnd(List children) { + return children.stream().reduce((first, second) -> first && second).get(); + } + + @Override + public Boolean visitOr(List children) { + return false; + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java new file mode 100644 index 0000000000..f3007c8aaf --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java @@ -0,0 +1,68 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.InternalRow; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link CompoundPredicate.Function} to eval or. */ +public class Or extends CompoundPredicate.Function { + + private static final long serialVersionUID = 1L; + + public static final Or INSTANCE = new Or(); + + private Or() {} + + @Override + public boolean test(InternalRow row, List children) { + for (Predicate child : children) { + if (child.test(row)) { + return true; + } + } + return false; + } + + @Override + public Optional negate(List children) { + List negatedChildren = new ArrayList<>(); + for (Predicate child : children) { + Optional negatedChild = child.negate(); + if (negatedChild.isPresent()) { + negatedChildren.add(negatedChild.get()); + } else { + return Optional.empty(); + } + } + return Optional.of(new CompoundPredicate(And.INSTANCE, negatedChildren)); + } + + @Override + public T visit(FunctionVisitor visitor, List children) { + return visitor.visitOr(children); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java new file mode 100644 index 0000000000..3e399d4fce --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java @@ -0,0 +1,52 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.List; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** Visit the predicate and check if it only contains partition key's predicate. */ +public class PartitionPredicateVisitor implements PredicateVisitor { + + private final List partitionKeys; + + public PartitionPredicateVisitor(List partitionKeys) { + this.partitionKeys = partitionKeys; + } + + @Override + public Boolean visit(LeafPredicate predicate) { + return partitionKeys.contains(predicate.fieldName()); + } + + @Override + public Boolean visit(CompoundPredicate predicate) { + for (Predicate child : predicate.children()) { + Boolean matched = child.visit(this); + + if (!matched) { + return false; + } + } + return true; + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java new file mode 100644 index 0000000000..55b6e3d2c0 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java @@ -0,0 +1,49 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.InternalRow; + +import java.io.Serializable; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * Predicate which returns Boolean and provides testing by stats. + * + * @see PredicateBuilder + * @since 0.4.0 + */ +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); + + /** @return the negation predicate of this predicate if possible. */ + Optional negate(); + + T visit(PredicateVisitor visitor); +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java new file mode 100644 index 0000000000..8a25715b7d --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java @@ -0,0 +1,446 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.row.Decimal; +import com.alibaba.fluss.row.TimestampLtz; +import com.alibaba.fluss.row.TimestampNtz; +import com.alibaba.fluss.types.DataField; +import com.alibaba.fluss.types.DataType; +import com.alibaba.fluss.types.DecimalType; +import com.alibaba.fluss.types.RowType; +import com.alibaba.fluss.utils.Preconditions; + +import javax.annotation.Nullable; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.alibaba.fluss.utils.TypeUtils.castFromString; +import static java.util.Collections.singletonList; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * A utility class to create {@link Predicate} object for common filter conditions. + * + * @since 0.4.0 + */ +public class PredicateBuilder { + + private final RowType rowType; + private final List fieldNames; + + public PredicateBuilder(RowType rowType) { + this.rowType = rowType; + this.fieldNames = rowType.getFieldNames(); + } + + public int indexOf(String field) { + return fieldNames.indexOf(field); + } + + public Predicate equal(int idx, Object literal) { + return leaf(Equal.INSTANCE, idx, literal); + } + + public Predicate notEqual(int idx, Object literal) { + return leaf(NotEqual.INSTANCE, idx, literal); + } + + public Predicate lessThan(int idx, Object literal) { + return leaf(LessThan.INSTANCE, idx, literal); + } + + public Predicate lessOrEqual(int idx, Object literal) { + return leaf(LessOrEqual.INSTANCE, idx, literal); + } + + public Predicate greaterThan(int idx, Object literal) { + return leaf(GreaterThan.INSTANCE, idx, literal); + } + + public Predicate greaterOrEqual(int idx, Object literal) { + return leaf(GreaterOrEqual.INSTANCE, idx, literal); + } + + public Predicate isNull(int idx) { + return leaf(IsNull.INSTANCE, idx); + } + + public Predicate isNotNull(int idx) { + return leaf(IsNotNull.INSTANCE, idx); + } + + 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 contains(int idx, Object patternLiteral) { + return leaf(Contains.INSTANCE, idx, patternLiteral); + } + + public Predicate leaf(NullFalseLeafBinaryFunction function, int idx, Object literal) { + DataField field = rowType.getFields().get(idx); + return new LeafPredicate( + function, field.getType(), idx, field.getName(), singletonList(literal)); + } + + public Predicate leaf(LeafUnaryFunction function, int idx) { + DataField field = rowType.getFields().get(idx); + return new LeafPredicate( + function, field.getType(), idx, field.getName(), Collections.emptyList()); + } + + public Predicate in(int idx, List literals) { + // In the IN predicate, 20 literals are critical for performance. + // If there are more than 20 literals, the performance will decrease. + if (literals.size() > 20) { + DataField field = rowType.getFields().get(idx); + return new LeafPredicate(In.INSTANCE, field.getType(), idx, field.getName(), literals); + } + + List equals = new ArrayList<>(literals.size()); + for (Object literal : literals) { + equals.add(equal(idx, literal)); + } + return or(equals); + } + + public Predicate notIn(int idx, List literals) { + return in(idx, literals).negate().get(); + } + + public Predicate between(int idx, Object includedLowerBound, Object includedUpperBound) { + return new CompoundPredicate( + And.INSTANCE, + Arrays.asList( + greaterOrEqual(idx, includedLowerBound), + lessOrEqual(idx, includedUpperBound))); + } + + public static Predicate and(Predicate... predicates) { + return and(Arrays.asList(predicates)); + } + + public static Predicate and(List predicates) { + Preconditions.checkArgument( + predicates.size() > 0, + "There must be at least 1 inner predicate to construct an AND predicate"); + if (predicates.size() == 1) { + return predicates.get(0); + } + return predicates.stream() + .reduce((a, b) -> new CompoundPredicate(And.INSTANCE, Arrays.asList(a, b))) + .get(); + } + + @Nullable + public static Predicate andNullable(Predicate... predicates) { + return andNullable(Arrays.asList(predicates)); + } + + @Nullable + public static Predicate andNullable(List predicates) { + predicates = predicates.stream().filter(Objects::nonNull).collect(Collectors.toList()); + if (predicates.isEmpty()) { + return null; + } + + return and(predicates); + } + + public static Predicate or(Predicate... predicates) { + return or(Arrays.asList(predicates)); + } + + public static Predicate or(List predicates) { + Preconditions.checkArgument( + predicates.size() > 0, + "There must be at least 1 inner predicate to construct an OR predicate"); + return predicates.stream() + .reduce((a, b) -> new CompoundPredicate(Or.INSTANCE, Arrays.asList(a, b))) + .get(); + } + + public static List splitAnd(@Nullable Predicate predicate) { + if (predicate == null) { + return Collections.emptyList(); + } + List result = new ArrayList<>(); + splitCompound(And.INSTANCE, predicate, result); + return result; + } + + public static List splitOr(@Nullable Predicate predicate) { + if (predicate == null) { + return Collections.emptyList(); + } + List result = new ArrayList<>(); + splitCompound(Or.INSTANCE, predicate, result); + return result; + } + + private static void splitCompound( + CompoundPredicate.Function function, Predicate predicate, List result) { + if (predicate instanceof CompoundPredicate + && ((CompoundPredicate) predicate).function().equals(function)) { + for (Predicate child : ((CompoundPredicate) predicate).children()) { + splitCompound(function, child, result); + } + } else { + result.add(predicate); + } + } + + public static Object convertJavaObject(DataType literalType, Object o) { + if (o == null) { + return null; + } + switch (literalType.getTypeRoot()) { + case BOOLEAN: + return o; + case BIGINT: + return ((Number) o).longValue(); + case DOUBLE: + return ((Number) o).doubleValue(); + case TINYINT: + return ((Number) o).byteValue(); + case SMALLINT: + return ((Number) o).shortValue(); + case INTEGER: + return ((Number) o).intValue(); + case FLOAT: + return ((Number) o).floatValue(); + case STRING: + return BinaryString.fromString(o.toString()); + case DATE: + // Hive uses `java.sql.Date.valueOf(lit.toString());` to convert a literal to Date + // Which uses `java.util.Date()` internally to create the object and that uses the + // TimeZone.getDefaultRef() + // To get back the expected date we have to use the LocalDate which gets rid of the + // TimeZone misery as it uses the year/month/day to generate the object + LocalDate localDate; + if (o instanceof java.sql.Timestamp) { + localDate = ((java.sql.Timestamp) o).toLocalDateTime().toLocalDate(); + } else if (o instanceof Date) { + localDate = ((Date) o).toLocalDate(); + } else if (o instanceof LocalDate) { + localDate = (LocalDate) o; + } else { + throw new UnsupportedOperationException( + "Unexpected date literal of class " + o.getClass().getName()); + } + LocalDate epochDay = + Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC).toLocalDate(); + return (int) ChronoUnit.DAYS.between(epochDay, localDate); + case TIME_WITHOUT_TIME_ZONE: + LocalTime localTime; + if (o instanceof java.sql.Time) { + localTime = ((java.sql.Time) o).toLocalTime(); + } else if (o instanceof LocalTime) { + localTime = (LocalTime) o; + } else { + throw new UnsupportedOperationException( + "Unexpected time literal of class " + o.getClass().getName()); + } + // return millis of a day + return (int) (localTime.toNanoOfDay() / 1_000_000); + case DECIMAL: + DecimalType decimalType = (DecimalType) literalType; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + return Decimal.fromBigDecimal((BigDecimal) o, precision, scale); + case TIMESTAMP_WITHOUT_TIME_ZONE: + if (o instanceof java.sql.Timestamp) { + LocalDateTime localDateTime = + ((Timestamp) o) + .toInstant() + .atZone(ZoneId.systemDefault()) + .toLocalDateTime(); + return TimestampNtz.fromLocalDateTime(localDateTime); + } else if (o instanceof Instant) { + Instant o1 = (Instant) o; + LocalDateTime dateTime = o1.atZone(ZoneId.systemDefault()).toLocalDateTime(); + return TimestampNtz.fromLocalDateTime(dateTime); + } else if (o instanceof LocalDateTime) { + return TimestampNtz.fromLocalDateTime((LocalDateTime) o); + } else { + throw new UnsupportedOperationException( + String.format( + "Unsupported class %s for timestamp without timezone ", + o.getClass())); + } + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + if (o instanceof java.sql.Timestamp) { + java.sql.Timestamp timestamp = (java.sql.Timestamp) o; + return TimestampLtz.fromInstant(timestamp.toInstant()); + } else if (o instanceof Instant) { + return TimestampLtz.fromInstant((Instant) o); + } else { + throw new UnsupportedOperationException( + String.format( + "Unsupported class %s for timestamp with local time zone ", + o.getClass())); + } + default: + throw new UnsupportedOperationException( + "Unsupported predicate leaf type " + literalType.getTypeRoot().name()); + } + } + + public static List pickTransformFieldMapping( + List predicates, List inputFields, List pickedFields) { + return pickTransformFieldMapping( + predicates, inputFields.stream().mapToInt(pickedFields::indexOf).toArray()); + } + + public static List pickTransformFieldMapping( + List predicates, int[] fieldIdxMapping) { + List pick = new ArrayList<>(); + for (Predicate p : predicates) { + Optional mapped = transformFieldMapping(p, fieldIdxMapping); + mapped.ifPresent(pick::add); + } + return pick; + } + + public static Optional transformFieldMapping( + Predicate predicate, int[] fieldIdxMapping) { + if (predicate instanceof CompoundPredicate) { + CompoundPredicate compoundPredicate = (CompoundPredicate) predicate; + List children = new ArrayList<>(); + for (Predicate child : compoundPredicate.children()) { + Optional mapped = transformFieldMapping(child, fieldIdxMapping); + if (mapped.isPresent()) { + children.add(mapped.get()); + } else { + return Optional.empty(); + } + } + return Optional.of(new CompoundPredicate(compoundPredicate.function(), children)); + } else { + LeafPredicate leafPredicate = (LeafPredicate) predicate; + int mapped = fieldIdxMapping[leafPredicate.index()]; + if (mapped >= 0) { + return Optional.of( + new LeafPredicate( + leafPredicate.function(), + leafPredicate.type(), + mapped, + leafPredicate.fieldName(), + leafPredicate.literals())); + } else { + return Optional.empty(); + } + } + } + + public static boolean containsFields(Predicate predicate, Set fields) { + if (predicate instanceof CompoundPredicate) { + for (Predicate child : ((CompoundPredicate) predicate).children()) { + if (containsFields(child, fields)) { + return true; + } + } + return false; + } else { + LeafPredicate leafPredicate = (LeafPredicate) predicate; + return fields.contains(leafPredicate.fieldName()); + } + } + + public static List excludePredicateWithFields( + @Nullable List predicates, Set fields) { + if (predicates == null || predicates.isEmpty() || fields.isEmpty()) { + return predicates; + } + return predicates.stream() + .filter(f -> !containsFields(f, fields)) + .collect(Collectors.toList()); + } + + @Nullable + public static Predicate partition( + Map map, RowType rowType, String defaultPartValue) { + Map internalValues = convertSpecToInternal(map, rowType, defaultPartValue); + List fieldNames = rowType.getFieldNames(); + Predicate predicate = null; + PredicateBuilder builder = new PredicateBuilder(rowType); + for (Map.Entry entry : internalValues.entrySet()) { + int idx = fieldNames.indexOf(entry.getKey()); + Object literal = internalValues.get(entry.getKey()); + Predicate predicateTemp = + literal == null ? builder.isNull(idx) : builder.equal(idx, literal); + if (predicate == null) { + predicate = predicateTemp; + } else { + predicate = PredicateBuilder.and(predicate, predicateTemp); + } + } + return predicate; + } + + public static Predicate partitions( + List> partitions, RowType rowType, String defaultPartValue) { + return PredicateBuilder.or( + partitions.stream() + .map(p -> PredicateBuilder.partition(p, rowType, defaultPartValue)) + .toArray(Predicate[]::new)); + } + + public static Map convertSpecToInternal( + Map spec, RowType partType, String defaultPartValue) { + Map partValues = new LinkedHashMap<>(); + for (Map.Entry entry : spec.entrySet()) { + partValues.put( + entry.getKey(), + defaultPartValue.equals(entry.getValue()) + ? null + : castFromString( + entry.getValue(), partType.getField(entry.getKey()).getType())); + } + return partValues; + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java new file mode 100644 index 0000000000..6b92039e10 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java @@ -0,0 +1,70 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link PredicateVisitor} which converts {@link Predicate} with projection. */ +public class PredicateProjectionConverter implements PredicateVisitor> { + + private final Map reversed; + + public PredicateProjectionConverter(int[] projection) { + this.reversed = new HashMap<>(); + for (int i = 0; i < projection.length; i++) { + reversed.put(projection[i], i); + } + } + + @Override + public Optional visit(LeafPredicate predicate) { + int index = predicate.index(); + Integer adjusted = reversed.get(index); + if (adjusted == null) { + return Optional.empty(); + } + + return Optional.of(predicate.copyWithNewIndex(adjusted)); + } + + @Override + public Optional visit(CompoundPredicate predicate) { + List converted = new ArrayList<>(); + boolean isAnd = predicate.function() instanceof And; + for (Predicate child : predicate.children()) { + Optional optional = child.visit(this); + if (optional.isPresent()) { + converted.add(optional.get()); + } else { + if (!isAnd) { + return Optional.empty(); + } + } + } + return Optional.of(new CompoundPredicate(predicate.function(), converted)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java new file mode 100644 index 0000000000..49989484b7 --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java @@ -0,0 +1,45 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A {@link PredicateVisitor} to replace {@link Predicate}. */ +public interface PredicateReplaceVisitor extends PredicateVisitor> { + + @Override + default Optional visit(CompoundPredicate predicate) { + List converted = new ArrayList<>(); + for (Predicate child : predicate.children()) { + Optional optional = child.visit(this); + if (optional.isPresent()) { + converted.add(optional.get()); + } else { + return Optional.empty(); + } + } + return Optional.of(new CompoundPredicate(predicate.function(), converted)); + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java new file mode 100644 index 0000000000..d5b1e049ba --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java @@ -0,0 +1,31 @@ +/* + * 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 com.alibaba.fluss.predicate; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** A visitor to visit {@link Predicate}. */ +public interface PredicateVisitor { + + T visit(LeafPredicate predicate); + + T visit(CompoundPredicate predicate); +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java new file mode 100644 index 0000000000..984a4a5dcd --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java @@ -0,0 +1,71 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.types.DataType; + +import java.util.List; +import java.util.Optional; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** + * A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like 'abc%' or filter like + * 'abc_'}. + */ +public class StartsWith extends NullFalseLeafBinaryFunction { + + public static final StartsWith INSTANCE = new StartsWith(); + + 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(); + } + + @Override + public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { + return visitor.visitStartsWith(fieldRef, literals.get(0)); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/types/RowType.java b/fluss-common/src/main/java/org/apache/fluss/types/RowType.java index 3c1122f63c..99262c1fd2 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/RowType.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/RowType.java @@ -265,4 +265,14 @@ public RowType build() { return new RowType(isNullable, fields); } } + + public DataField getField(String fieldName) { + for (DataField field : fields) { + if (field.getName().equals(fieldName)) { + return field; + } + } + + throw new RuntimeException("Cannot find field: " + fieldName); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java index 6275182c4c..8c61a10f6f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java @@ -17,9 +17,9 @@ package org.apache.fluss.utils; -import org.apache.fluss.row.BinaryString; -import org.apache.fluss.row.TimestampLtz; -import org.apache.fluss.row.TimestampNtz; +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.row.TimestampLtz; +import com.alibaba.fluss.row.TimestampNtz; import java.time.DateTimeException; import java.util.List; @@ -76,9 +76,9 @@ public static TimestampNtz toTimestampNtz(BinaryString input, int precision) return DateTimeUtils.parseTimestampData(input.toString(), precision); } - /** Used by {@code CAST(x as TIMESTAMP_LTZ)}. */ - public static TimestampLtz toTimestampLtz( - BinaryString input, int precision, TimeZone localTimeZone) throws DateTimeException { - return DateTimeUtils.parseTimestampData(input.toString(), precision, localTimeZone); + /** Used by {@code CAST(x as TIMESTAMPLTZ)}. */ + public static TimestampLtz toTimestampltz(BinaryString input, int precision, TimeZone timeZone) + throws DateTimeException { + return DateTimeUtils.parseTimestampData(input.toString(), precision, timeZone); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java index 9106993eae..fd645bb944 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java @@ -17,8 +17,8 @@ package org.apache.fluss.utils; -import org.apache.fluss.row.TimestampLtz; -import org.apache.fluss.row.TimestampNtz; +import com.alibaba.fluss.row.TimestampLtz; +import com.alibaba.fluss.row.TimestampNtz; import java.time.DateTimeException; import java.time.LocalDate; diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java new file mode 100644 index 0000000000..1de02da0c8 --- /dev/null +++ b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java @@ -0,0 +1,86 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.GenericRow; +import com.alibaba.fluss.types.IntType; +import com.alibaba.fluss.types.RowType; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link PredicateBuilder}. */ +public class PredicateBuilderTest { + + @Test + 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); + } + + @Test + 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); + } + + @Test + public void testSplitAnd() { + PredicateBuilder builder = + new PredicateBuilder( + RowType.of( + new IntType(), + new IntType(), + new IntType(), + new IntType(), + new IntType(), + new IntType(), + new IntType())); + + Predicate child1 = + PredicateBuilder.or(builder.isNull(0), builder.isNull(1), builder.isNull(2)); + Predicate child2 = + PredicateBuilder.and(builder.isNull(3), builder.isNull(4), builder.isNull(5)); + Predicate child3 = builder.isNull(6); + + assertThat(PredicateBuilder.splitAnd(PredicateBuilder.and(child1, child2, child3))) + .isEqualTo( + Arrays.asList( + child1, + builder.isNull(3), + builder.isNull(4), + builder.isNull(5), + child3)); + } +} diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java new file mode 100644 index 0000000000..471992d263 --- /dev/null +++ b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java @@ -0,0 +1,408 @@ +/* + * 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 com.alibaba.fluss.predicate; + +import com.alibaba.fluss.row.GenericRow; +import com.alibaba.fluss.types.IntType; +import com.alibaba.fluss.types.RowType; +import com.alibaba.fluss.types.StringType; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static com.alibaba.fluss.row.BinaryString.fromString; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link Predicate}s. */ +public class PredicateTest { + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.notEqual(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.equal(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.lessOrEqual(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.lessThan(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.greaterOrEqual(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.greaterThan(0, 5)); + } + + @Test + 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); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.isNotNull(0)); + } + + @Test + 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(predicate.negate().orElse(null)).isEqualTo(builder.isNull(0)); + } + + @Test + public void testIn() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + 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); + } + + @Test + public void testInNull() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + 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); + } + + @Test + public void testNotIn() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + 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); + } + + @Test + public void testNotInNull() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + 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); + } + + @Test + public void testEndsWith() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); + 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); + } + + @Test + public void testLargeIn() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + List literals = new ArrayList<>(); + literals.add(1); + literals.add(3); + for (int i = 10; i < 30; i++) { + literals.add(i); + } + 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); + } + + @Test + public void testLargeInNull() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + List literals = new ArrayList<>(); + literals.add(1); + literals.add(null); + literals.add(3); + for (int i = 10; i < 30; i++) { + literals.add(i); + } + 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); + } + + @Test + public void testLargeNotIn() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + List literals = new ArrayList<>(); + literals.add(1); + literals.add(3); + for (int i = 10; i < 30; i++) { + literals.add(i); + } + 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); + } + + @Test + public void testLargeNotInNull() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + List literals = new ArrayList<>(); + literals.add(1); + literals.add(null); + literals.add(3); + for (int i = 10; i < 30; i++) { + literals.add(i); + } + 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); + } + + @Test + 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(predicate.negate().orElse(null)) + .isEqualTo(PredicateBuilder.or(builder.notEqual(0, 3), builder.notEqual(1, 5))); + } + + @Test + 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(predicate.negate().orElse(null)) + .isEqualTo(PredicateBuilder.and(builder.notEqual(0, 3), builder.notEqual(1, 5))); + } + + @Test + public void testUnknownStats() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); + Predicate predicate = builder.equal(0, 5); + } + + @Test + public void testPredicateToString() { + PredicateBuilder builder1 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p1 = builder1.equal(0, 5); + assertThat(p1.toString()).isEqualTo("Equal(f0, 5)"); + + PredicateBuilder builder2 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p2 = builder2.greaterThan(0, 5); + assertThat(p2.toString()).isEqualTo("GreaterThan(f0, 5)"); + + PredicateBuilder builder3 = new PredicateBuilder(RowType.of(new IntType(), new IntType())); + Predicate p3 = PredicateBuilder.and(builder3.equal(0, 3), builder3.equal(1, 5)); + assertThat(p3.toString()).isEqualTo("And([Equal(f0, 3), Equal(f1, 5)])"); + + PredicateBuilder builder4 = new PredicateBuilder(RowType.of(new IntType(), new IntType())); + Predicate p4 = PredicateBuilder.or(builder4.equal(0, 3), builder4.equal(1, 5)); + assertThat(p4.toString()).isEqualTo("Or([Equal(f0, 3), Equal(f1, 5)])"); + + PredicateBuilder builder5 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p5 = builder5.isNotNull(0); + assertThat(p5.toString()).isEqualTo("IsNotNull(f0)"); + + PredicateBuilder builder6 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p6 = builder6.in(0, Arrays.asList(1, null, 3, 4)); + assertThat(p6.toString()) + .isEqualTo( + "Or([Or([Or([Equal(f0, 1), Equal(f0, null)]), Equal(f0, 3)]), Equal(f0, 4)])"); + + PredicateBuilder builder7 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p7 = builder7.notIn(0, Arrays.asList(1, null, 3, 4)); + assertThat(p7.toString()) + .isEqualTo( + "And([And([And([NotEqual(f0, 1), NotEqual(f0, null)]), NotEqual(f0, 3)]), NotEqual(f0, 4)])"); + + PredicateBuilder builder8 = new PredicateBuilder(RowType.of(new IntType())); + List literals = new ArrayList<>(); + for (int i = 1; i <= 21; i++) { + literals.add(i); + } + Predicate p8 = builder8.in(0, literals); + assertThat(p8.toString()) + .isEqualTo( + "In(f0, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21])"); + + PredicateBuilder builder9 = new PredicateBuilder(RowType.of(new IntType())); + Predicate p9 = builder9.notIn(0, literals); + assertThat(p9.toString()) + .isEqualTo( + "NotIn(f0, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21])"); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 5739381d36..895169ec4c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -17,28 +17,23 @@ package org.apache.fluss.flink.source; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.FlinkConnectorOptions; -import org.apache.fluss.flink.source.deserializer.RowDataDeserializationSchema; -import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; -import org.apache.fluss.flink.source.lookup.FlinkAsyncLookupFunction; -import org.apache.fluss.flink.source.lookup.FlinkLookupFunction; -import org.apache.fluss.flink.source.lookup.LookupNormalizer; -import org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils; -import org.apache.fluss.flink.utils.FlinkConversions; -import org.apache.fluss.flink.utils.PushdownUtils; -import org.apache.fluss.flink.utils.PushdownUtils.FieldEqual; -import org.apache.fluss.lake.source.LakeSource; -import org.apache.fluss.lake.source.LakeSplit; -import org.apache.fluss.metadata.MergeEngineType; -import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.predicate.GreaterOrEqual; -import org.apache.fluss.predicate.LeafPredicate; -import org.apache.fluss.predicate.Predicate; -import org.apache.fluss.predicate.PredicateBuilder; -import org.apache.fluss.row.TimestampLtz; -import org.apache.fluss.types.DataTypes; -import org.apache.fluss.types.RowType; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.connector.flink.FlinkConnectorOptions; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer; +import com.alibaba.fluss.connector.flink.source.lookup.FlinkAsyncLookupFunction; +import com.alibaba.fluss.connector.flink.source.lookup.FlinkLookupFunction; +import com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer; +import com.alibaba.fluss.connector.flink.utils.FlinkConnectorOptionsUtils; +import com.alibaba.fluss.connector.flink.utils.FlinkConversions; +import com.alibaba.fluss.connector.flink.utils.PushdownUtils; +import com.alibaba.fluss.connector.flink.utils.PushdownUtils.ValueConversion; +import com.alibaba.fluss.metadata.MergeEngineType; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.predicate.PartitionPredicateVisitor; +import com.alibaba.fluss.predicate.Predicate; +import com.alibaba.fluss.predicate.PredicateBuilder; +import com.alibaba.fluss.predicate.PredicateVisitor; +import com.alibaba.fluss.types.RowType; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -72,8 +67,7 @@ import org.apache.flink.table.functions.LookupFunction; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.flink.table.types.logical.VarCharType; import javax.annotation.Nullable; @@ -85,6 +79,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; import static org.apache.fluss.flink.utils.LakeSourceUtils.createLakeSource; import static org.apache.fluss.flink.utils.PushdownUtils.ValueConversion.FLINK_INTERNAL_VALUE; @@ -144,11 +140,7 @@ public class FlinkTableSource private long limit = -1; - private List partitionFilters = Collections.emptyList(); - - private final Map tableOptions; - - @Nullable private LakeSource lakeSource; + @Nullable protected Predicate predicate; public FlinkTableSource( TablePath tablePath, @@ -308,10 +300,8 @@ public boolean isBounded() { projectedFields, offsetsInitializer, scanPartitionDiscoveryIntervalMs, - new RowDataDeserializationSchema(), streaming, - partitionFilters, - enableLakeSource ? lakeSource : null); + predicate); if (!streaming) { // return a bounded source provide to make planner happy, @@ -475,18 +465,19 @@ public Result applyFilters(List filters) { && startupOptions.startupMode == FlinkConnectorOptions.ScanStartupMode.FULL && hasPrimaryKey() && filters.size() == primaryKeyIndexes.length) { + Map primaryKeyTypes = getPrimaryKeyTypes(); - List fieldEquals = - extractFieldEquals( + List fieldEquals = + PushdownUtils.extractFieldEquals( filters, primaryKeyTypes, acceptedFilters, remainingFilters, - FLINK_INTERNAL_VALUE); + ValueConversion.FLINK_INTERNAL_VALUE); int[] keyRowProjection = getKeyRowProjection(); HashSet visitedPkFields = new HashSet<>(); GenericRowData lookupRow = new GenericRowData(primaryKeyIndexes.length); - for (FieldEqual fieldEqual : fieldEquals) { + for (PushdownUtils.FieldEqual fieldEqual : fieldEquals) { lookupRow.setField(keyRowProjection[fieldEqual.fieldIndex], fieldEqual.equalValue); visitedPkFields.add(fieldEqual.fieldIndex); } @@ -497,51 +488,46 @@ && hasPrimaryKey() singleRowFilter = lookupRow; return Result.of(acceptedFilters, remainingFilters); } else if (isPartitioned()) { - // dynamic partition pushdown - List fieldEquals = - extractFieldEquals( - filters, - getPartitionKeyTypes(), - acceptedFilters, - remainingFilters, - FLUSS_INTERNAL_VALUE); - - // partitions are filtered by string representations, convert the equals to string first - partitionFilters = stringifyFieldEquals(fieldEquals); - - // lake source is not null - if (lakeSource != null) { - // and exist field equals, push down to lake source - if (!fieldEquals.isEmpty()) { - // convert flink row type to fluss row type - RowType flussRowType = FlinkConversions.toFlussRowType(tableOutputType); - - List lakePredicates = new ArrayList<>(); - PredicateBuilder predicateBuilder = new PredicateBuilder(flussRowType); - - for (FieldEqual fieldEqual : fieldEquals) { - lakePredicates.add( - predicateBuilder.equal( - fieldEqual.fieldIndex, fieldEqual.equalValue)); - } - - if (!lakePredicates.isEmpty()) { - final LakeSource.FilterPushDownResult filterPushDownResult = - lakeSource.withFilters(lakePredicates); - if (filterPushDownResult.acceptedPredicates().size() - != lakePredicates.size()) { - LOG.info( - "LakeSource rejected some partition filters. Falling back to Flink-side filtering."); - // Flink will apply all filters to preserve correctness - return Result.of(Collections.emptyList(), filters); - } + // apply partition filter pushdown + List converted = new ArrayList<>(); + + List fieldNames = tableOutputType.getFieldNames(); + List partitionKeys = + Arrays.stream(partitionKeyIndexes) + .mapToObj(fieldNames::get) + .collect(Collectors.toList()); + + PredicateVisitor partitionPredicateVisitor = + new PartitionPredicateVisitor(partitionKeys); + LogicalType[] partitionKeyTypes = + partitionKeys.stream() + .map(key -> VarCharType.STRING_TYPE) + .toArray(LogicalType[]::new); + for (ResolvedExpression filter : filters) { + + Optional predicateOptional = + PredicateConverter.convert( + org.apache.flink.table.types.logical.RowType.of( + partitionKeyTypes, partitionKeys.toArray(new String[0])), + filter); + + if (!predicateOptional.isPresent()) { + remainingFilters.add(filter); + } else { + Predicate p = predicateOptional.get(); + if (!p.visit(partitionPredicateVisitor)) { + remainingFilters.add(filter); + } else { + acceptedFilters.add(filter); } + converted.add(p); } } + predicate = converted.isEmpty() ? null : PredicateBuilder.and(converted); return Result.of(acceptedFilters, remainingFilters); - } else { - return Result.of(Collections.emptyList(), filters); } + + return Result.of(acceptedFilters, remainingFilters); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index 867ff442f8..6e28eeab8c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -17,33 +17,33 @@ package org.apache.fluss.flink.source.enumerator; -import org.apache.fluss.client.Connection; -import org.apache.fluss.client.ConnectionFactory; -import org.apache.fluss.client.admin.Admin; -import org.apache.fluss.client.metadata.KvSnapshots; -import org.apache.fluss.config.ConfigOptions; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.lake.LakeSplitGenerator; -import org.apache.fluss.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; -import org.apache.fluss.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; -import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; -import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; -import org.apache.fluss.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; -import org.apache.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; -import org.apache.fluss.flink.source.event.PartitionsRemovedEvent; -import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; -import org.apache.fluss.flink.source.split.LogSplit; -import org.apache.fluss.flink.source.split.SourceSplitBase; -import org.apache.fluss.flink.source.state.SourceEnumeratorState; -import org.apache.fluss.flink.utils.PushdownUtils.FieldEqual; -import org.apache.fluss.lake.source.LakeSource; -import org.apache.fluss.lake.source.LakeSplit; -import org.apache.fluss.metadata.PartitionInfo; -import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.types.DataField; -import org.apache.fluss.utils.ExceptionUtils; +import com.alibaba.fluss.client.Connection; +import com.alibaba.fluss.client.ConnectionFactory; +import com.alibaba.fluss.client.admin.Admin; +import com.alibaba.fluss.client.metadata.KvSnapshots; +import com.alibaba.fluss.config.ConfigOptions; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.connector.flink.lakehouse.LakeSplitGenerator; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; +import com.alibaba.fluss.connector.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; +import com.alibaba.fluss.connector.flink.source.event.PartitionBucketsUnsubscribedEvent; +import com.alibaba.fluss.connector.flink.source.event.PartitionsRemovedEvent; +import com.alibaba.fluss.connector.flink.source.split.HybridSnapshotLogSplit; +import com.alibaba.fluss.connector.flink.source.split.LogSplit; +import com.alibaba.fluss.connector.flink.source.split.SourceSplitBase; +import com.alibaba.fluss.connector.flink.source.state.SourceEnumeratorState; +import com.alibaba.fluss.metadata.PartitionInfo; +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TableInfo; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.predicate.Predicate; +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.row.GenericRow; +import com.alibaba.fluss.row.InternalRow; +import com.alibaba.fluss.utils.ExceptionUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.source.SourceEvent; @@ -131,11 +131,7 @@ public class FlinkSourceEnumerator private boolean lakeEnabled = false; - private volatile boolean closed = false; - - private final List partitionFilters; - - @Nullable private final LakeSource lakeSource; + private Predicate predicate; public FlinkSourceEnumerator( TablePath tablePath, @@ -146,18 +142,20 @@ public FlinkSourceEnumerator( OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, boolean streaming, - List partitionFilters) { + Predicate predicate) { this( tablePath, flussConf, hasPrimaryKey, isPartitioned, context, + Collections.emptySet(), + Collections.emptyMap(), + null, startingOffsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters, - null); + predicate); } public FlinkSourceEnumerator( @@ -166,41 +164,38 @@ public FlinkSourceEnumerator( boolean hasPrimaryKey, boolean isPartitioned, SplitEnumeratorContext context, + Set assignedTableBuckets, + Map assignedPartitions, + List pendingHybridLakeFlussSplits, OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, - boolean streaming, - List partitionFilters, - @Nullable LakeSource lakeSource) { + boolean streaming) { this( tablePath, flussConf, - hasPrimaryKey, isPartitioned, + hasPrimaryKey, context, - Collections.emptySet(), - Collections.emptyMap(), - null, + assignedTableBuckets, + assignedPartitions, startingOffsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters, - lakeSource); + null); } public FlinkSourceEnumerator( TablePath tablePath, Configuration flussConf, - boolean hasPrimaryKey, boolean isPartitioned, + boolean hasPrimaryKey, SplitEnumeratorContext context, Set assignedTableBuckets, Map assignedPartitions, - List pendingHybridLakeFlussSplits, OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, boolean streaming, - List partitionFilters, - @Nullable LakeSource lakeSource) { + Predicate predicate) { this.tablePath = checkNotNull(tablePath); this.flussConf = checkNotNull(flussConf); this.hasPrimaryKey = hasPrimaryKey; @@ -219,7 +214,7 @@ public FlinkSourceEnumerator( this.partitionFilters = checkNotNull(partitionFilters); this.stoppingOffsetsInitializer = streaming ? new NoStoppingOffsetsInitializer() : OffsetsInitializer.latest(); - this.lakeSource = lakeSource; + this.predicate = predicate; } @Override @@ -338,13 +333,27 @@ private List initNonPartitionedSplits() { } private Set listPartitions() { - if (closed) { - return Collections.emptySet(); - } + try { List partitionInfos = flussAdmin.listPartitionInfos(tablePath).get(); - partitionInfos = applyPartitionFilter(partitionInfos); - return new LinkedHashSet<>(partitionInfos); + if (predicate == null) { + return new HashSet<>(partitionInfos); + } else { + Set filteredPartitionInfos = + partitionInfos.stream() + .filter( + partitionInfo -> + predicate.test( + convertPartitionInfoToInternalRow( + partitionInfo))) + .collect(Collectors.toSet()); + LOG.info( + "Filtered partitions {} for table {} with predicate: {}", + filteredPartitionInfos, + tablePath, + predicate); + return filteredPartitionInfos; + } } catch (Exception e) { throw new FlinkRuntimeException( String.format("Failed to list partitions for %s", tablePath), @@ -352,32 +361,8 @@ private Set listPartitions() { } } - /** Apply partition filter. */ - private List applyPartitionFilter(List partitionInfos) { - if (!partitionFilters.isEmpty()) { - return partitionInfos.stream() - .filter( - partitionInfo -> { - Map specMap = - partitionInfo.getPartitionSpec().getSpecMap(); - // use getFields() instead of getFieldNames() to - // avoid collection construction - List fields = tableInfo.getRowType().getFields(); - for (FieldEqual filter : partitionFilters) { - String fieldName = fields.get(filter.fieldIndex).getName(); - String partitionValue = specMap.get(fieldName); - if (partitionValue == null - || !filter.equalValue - .toString() - .equals(partitionValue)) { - return false; - } - } - return true; - }) - .collect(Collectors.toList()); - } - return partitionInfos; + private InternalRow convertPartitionInfoToInternalRow(PartitionInfo partitionInfo) { + return GenericRow.of(BinaryString.fromString(partitionInfo.getPartitionName())); } /** Init the splits for Fluss. */ diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 24757cb05f..3cc416618b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -708,6 +708,39 @@ void testReadTimestampGreaterThanMaxTimestamp() throws Exception { } } + @Test + void testStreamingReadPartitionPushDown() throws Exception { + + tEnv.executeSql( + "create table partitioned_table" + + " (a int not null, b varchar, c string, primary key (a, c) NOT ENFORCED) partitioned by (c) " + + "with ('table.auto-partition.enabled' = 'true', 'table.auto-partition.time-unit' = 'year')"); + TablePath tablePath = TablePath.of(DEFAULT_DB, "partitioned_table"); + + // write data into partitions and wait snapshot is done + Map partitionNameById = + waitUntilPartitions(FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath); + List expectedRowValues = + writeRowsToPartition(tablePath, partitionNameById.values()).stream() + .filter(s -> s.contains("2025")) + .collect(Collectors.toList()); + waitUtilAllBucketFinishSnapshot(admin, tablePath, partitionNameById.values()); + + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql("select * from partitioned_table where c in( '2000','2001','2025')") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, false); + + // then create some new partitions, and write rows to the new partitions + List newPartitions = Arrays.asList("2000", "2001"); + FlinkTestBase.createPartitions( + FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath, newPartitions); + // write data to the new partitions + expectedRowValues = writeRowsToPartition(tablePath, newPartitions); + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + // ------------------------------------------------------------------------------------- // Fluss look source tests // ------------------------------------------------------------------------------------- diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index 7218f38913..b4ba22deae 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -97,7 +97,7 @@ void testPkTableNoSnapshotSplits() throws Throwable { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList()); + null); enumerator.start(); @@ -144,7 +144,7 @@ void testPkTableWithSnapshotSplits() throws Throwable { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList()); + null); enumerator.start(); // register all read for (int i = 0; i < numSubtasks; i++) { @@ -215,7 +215,7 @@ void testNonPkTable() throws Throwable { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList()); + null); enumerator.start(); @@ -261,7 +261,7 @@ void testReaderRegistrationTriggerAssignments() throws Throwable { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList()); + null); enumerator.start(); @@ -297,7 +297,7 @@ void testAddSplitBack() throws Throwable { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList()); + null); enumerator.start(); @@ -357,7 +357,6 @@ void testRestore() throws Throwable { OffsetsInitializer.earliest(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList(), null); enumerator.start(); @@ -401,7 +400,7 @@ void testDiscoverPartitionsPeriodically(boolean isPrimaryKeyTable) throws Throwa OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList())) { + null)) { Map partitionNameByIds = waitUntilPartitions(zooKeeperClient, DEFAULT_TABLE_PATH); enumerator.start(); @@ -516,7 +515,7 @@ void testGetSplitOwner() throws Exception { OffsetsInitializer.full(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, - Collections.emptyList())) { + null)) { // test splits for same non-partitioned bucket, should assign to same task TableBucket t1 = new TableBucket(tableId, 0); From a9f2ab1272b6ca5768083f9a2509fc70d4d0b7fd Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 19 Feb 2025 17:26:31 +0800 Subject: [PATCH 02/22] [flink]remove useless visitor --- .../fluss/predicate/InPredicateVisitor.java | 57 -------- .../OnlyPartitionKeyEqualVisitor.java | 124 ------------------ .../predicate/PredicateReplaceVisitor.java | 45 ------- 3 files changed, 226 deletions(-) delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java deleted file mode 100644 index 5cd87ebc1a..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/InPredicateVisitor.java +++ /dev/null @@ -1,57 +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 com.alibaba.fluss.predicate; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A utils to handle {@link Predicate}. */ -public class InPredicateVisitor { - - /** - * Method for handling with In CompoundPredicate. - * - * @param predicate CompoundPredicate to traverse handle - * @param leafName LeafPredicate name - */ - public static Optional> extractInElements(Predicate predicate, String leafName) { - if (!(predicate instanceof CompoundPredicate)) { - return Optional.empty(); - } - - CompoundPredicate compoundPredicate = (CompoundPredicate) predicate; - List leafValues = new ArrayList<>(); - List children = compoundPredicate.children(); - for (Predicate leaf : children) { - if (leaf instanceof LeafPredicate - && (((LeafPredicate) leaf).function() instanceof Equal) - && leaf.visit(LeafPredicateExtractor.INSTANCE).get(leafName) != null) { - leafValues.add(((LeafPredicate) leaf).literals().get(0)); - } else { - return Optional.empty(); - } - } - return Optional.of(leafValues); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java deleted file mode 100644 index 80913446f0..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/OnlyPartitionKeyEqualVisitor.java +++ /dev/null @@ -1,124 +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 com.alibaba.fluss.predicate; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Visit the predicate and check if it only contains partition keys and can be push down. */ -public class OnlyPartitionKeyEqualVisitor implements FunctionVisitor { - - private final List partitionKeys; - - private final Map partitions; - - public OnlyPartitionKeyEqualVisitor(List partitionKeys) { - this.partitionKeys = partitionKeys; - partitions = new HashMap<>(); - } - - public Map partitions() { - return partitions; - } - - @Override - public Boolean visitIsNotNull(FieldRef fieldRef) { - return false; - } - - @Override - public Boolean visitIsNull(FieldRef fieldRef) { - return false; - } - - @Override - public Boolean visitStartsWith(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitEndsWith(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitContains(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitLessThan(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitGreaterOrEqual(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitNotEqual(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitLessOrEqual(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitEqual(FieldRef fieldRef, Object literal) { - boolean contains = partitionKeys.contains(fieldRef.name()); - if (contains) { - partitions.put(fieldRef.name(), literal.toString()); - return true; - } - return false; - } - - @Override - public Boolean visitGreaterThan(FieldRef fieldRef, Object literal) { - return false; - } - - @Override - public Boolean visitIn(FieldRef fieldRef, List literals) { - return false; - } - - @Override - public Boolean visitNotIn(FieldRef fieldRef, List literals) { - return false; - } - - @Override - public Boolean visitAnd(List children) { - return children.stream().reduce((first, second) -> first && second).get(); - } - - @Override - public Boolean visitOr(List children) { - return false; - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java deleted file mode 100644 index 49989484b7..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateReplaceVisitor.java +++ /dev/null @@ -1,45 +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 com.alibaba.fluss.predicate; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link PredicateVisitor} to replace {@link Predicate}. */ -public interface PredicateReplaceVisitor extends PredicateVisitor> { - - @Override - default Optional visit(CompoundPredicate predicate) { - List converted = new ArrayList<>(); - for (Predicate child : predicate.children()) { - Optional optional = child.visit(this); - if (optional.isPresent()) { - converted.add(optional.get()); - } else { - return Optional.empty(); - } - } - return Optional.of(new CompoundPredicate(predicate.function(), converted)); - } -} From 940bb4fd4821b61ae408ee335ca39a51ac0ad28f Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:42:28 +0800 Subject: [PATCH 03/22] [flink]fix and add log --- .../predicate/LeafPredicateExtractor.java | 48 ------------- .../PredicateProjectionConverter.java | 70 ------------------- 2 files changed, 118 deletions(-) delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java deleted file mode 100644 index 0d6375e3f1..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicateExtractor.java +++ /dev/null @@ -1,48 +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 com.alibaba.fluss.predicate; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Extract leaf predicate for field names. */ -public class LeafPredicateExtractor implements PredicateVisitor> { - - public static final LeafPredicateExtractor INSTANCE = new LeafPredicateExtractor(); - - @Override - public Map visit(LeafPredicate predicate) { - return Collections.singletonMap(predicate.fieldName(), predicate); - } - - @Override - public Map visit(CompoundPredicate predicate) { - if (predicate.function() instanceof And) { - Map leafPredicates = new HashMap<>(); - predicate.children().stream().map(p -> p.visit(this)).forEach(leafPredicates::putAll); - return leafPredicates; - } - return Collections.emptyMap(); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java deleted file mode 100644 index 6b92039e10..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateProjectionConverter.java +++ /dev/null @@ -1,70 +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 com.alibaba.fluss.predicate; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link PredicateVisitor} which converts {@link Predicate} with projection. */ -public class PredicateProjectionConverter implements PredicateVisitor> { - - private final Map reversed; - - public PredicateProjectionConverter(int[] projection) { - this.reversed = new HashMap<>(); - for (int i = 0; i < projection.length; i++) { - reversed.put(projection[i], i); - } - } - - @Override - public Optional visit(LeafPredicate predicate) { - int index = predicate.index(); - Integer adjusted = reversed.get(index); - if (adjusted == null) { - return Optional.empty(); - } - - return Optional.of(predicate.copyWithNewIndex(adjusted)); - } - - @Override - public Optional visit(CompoundPredicate predicate) { - List converted = new ArrayList<>(); - boolean isAnd = predicate.function() instanceof And; - for (Predicate child : predicate.children()) { - Optional optional = child.visit(this); - if (optional.isPresent()) { - converted.add(optional.get()); - } else { - if (!isAnd) { - return Optional.empty(); - } - } - } - return Optional.of(new CompoundPredicate(predicate.function(), converted)); - } -} From 0b2330cd7d7a780afb4e4d73730ccd82c260a724 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 3 Mar 2025 11:23:03 +0800 Subject: [PATCH 04/22] [flink]remove useless code --- .../com/alibaba/fluss/predicate/PredicateBuilder.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java index 8a25715b7d..83d5c5bc01 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java @@ -26,7 +26,7 @@ import com.alibaba.fluss.types.DataType; import com.alibaba.fluss.types.DecimalType; import com.alibaba.fluss.types.RowType; -import com.alibaba.fluss.utils.Preconditions; +import com.alibaba.fluss.utils.TypeUtils; import javax.annotation.Nullable; @@ -51,7 +51,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static com.alibaba.fluss.utils.TypeUtils.castFromString; +import static com.alibaba.fluss.utils.Preconditions.checkArgument; import static java.util.Collections.singletonList; /* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache @@ -165,7 +165,7 @@ public static Predicate and(Predicate... predicates) { } public static Predicate and(List predicates) { - Preconditions.checkArgument( + checkArgument( predicates.size() > 0, "There must be at least 1 inner predicate to construct an AND predicate"); if (predicates.size() == 1) { @@ -196,7 +196,7 @@ public static Predicate or(Predicate... predicates) { } public static Predicate or(List predicates) { - Preconditions.checkArgument( + checkArgument( predicates.size() > 0, "There must be at least 1 inner predicate to construct an OR predicate"); return predicates.stream() @@ -438,7 +438,7 @@ public static Map convertSpecToInternal( entry.getKey(), defaultPartValue.equals(entry.getValue()) ? null - : castFromString( + : TypeUtils.castFromString( entry.getValue(), partType.getField(entry.getKey()).getType())); } return partValues; From 3fd389373bf24d6111f107ebd1e3465f06b578af Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 3 Mar 2025 11:36:23 +0800 Subject: [PATCH 05/22] [common]Introduce Predicate to do filter and partition push down. --- .../src/main/java/com/alibaba/fluss/predicate/CompareUtils.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java index 5059e90f35..e2302af834 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java @@ -33,6 +33,8 @@ private CompareUtils() {} public static int compareLiteral(DataType type, Object v1, Object v2) { if (v1 instanceof Comparable) { + // because BinaryString can not serialize so v1 or v2 may be BinaryString convert to + // String for compare if (v1 instanceof BinaryString) { v1 = ((BinaryString) v1).toString(); } From 49e09d5f41208adcc8a432f618435c943f7c821c Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 3 Mar 2025 12:34:58 +0800 Subject: [PATCH 06/22] [common]jacoco exclude --- fluss-test-coverage/pom.xml | 53 +++++-------------------------------- 1 file changed, 6 insertions(+), 47 deletions(-) diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index aaa08c07db..a0a3b8e8c3 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -368,54 +368,13 @@ org.apache.fluss.metrics.* - org.apache.fluss.flink.lake.* - org.apache.fluss.kafka.* - - org.apache.fluss.tools.ci.* + + com.alibaba.fluss.lakehouse.* + com.alibaba.fluss.connector.flink.lakehouse.* + + com.alibaba.fluss.lakehouse.cli.* - org.apache.fluss.predicate.* - - org.apache.fluss.lake.source.* - - org.apache.fluss.dist.DummyClass - org.apache.fluss.flink.DummyClass120 - org.apache.fluss.lake.batch.ArrowRecordBatch - org.apache.fluss.lake.committer.CommittedLakeSnapshot - org.apache.fluss.lake.paimon.FlussDataTypeToPaimonDataType - - org.apache.fluss.lake.lance.* - - org.apache.fluss.lake.iceberg.* - org.apache.fluss.row.encode.iceberg.* - org.apache.fluss.bucketing.IcebergBucketingFunction - - org.apache.fluss.flink.tiering.source.TieringSourceOptions - org.apache.fluss.flink.tiering.source.TieringSource.Builder - org.apache.fluss.flink.tiering.source.TieringSource - - org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator - - - org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator.HeartBeatHelper - - org.apache.fluss.flink.tiering.source.TieringWriterInitContext - - org.apache.fluss.flink.tiering.source.TieringSourceReader - org.apache.fluss.flink.tiering.source.TableBucketWriteResultEmitter - - - org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo* - - - org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory - - org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo* - - - org.apache.fluss.flink.tiering.LakeTieringJobBuilder - - org.apache.fluss.flink.tiering.FlussLakeTieringEntrypoint - + com.alibaba.fluss.predicate.* From 2011c44e613d75c5c300cb85604a05a03a1cf1a9 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Fri, 20 Jun 2025 17:19:27 +0800 Subject: [PATCH 07/22] [common]Introduce Predicate to do filter and partition push down. --- .../java/com/alibaba/fluss/predicate/And.java | 15 +++++++-------- .../alibaba/fluss/predicate/CompareUtils.java | 15 +++++++-------- .../fluss/predicate/CompoundPredicate.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/Contains.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/EndsWith.java | 15 +++++++-------- .../java/com/alibaba/fluss/predicate/Equal.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/FieldRef.java | 15 +++++++-------- .../fluss/predicate/FunctionVisitor.java | 15 +++++++-------- .../alibaba/fluss/predicate/GreaterOrEqual.java | 15 +++++++-------- .../alibaba/fluss/predicate/GreaterThan.java | 15 +++++++-------- .../java/com/alibaba/fluss/predicate/In.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/IsNotNull.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/IsNull.java | 15 +++++++-------- .../alibaba/fluss/predicate/LeafFunction.java | 15 +++++++-------- .../alibaba/fluss/predicate/LeafPredicate.java | 15 +++++++-------- .../fluss/predicate/LeafUnaryFunction.java | 15 +++++++-------- .../alibaba/fluss/predicate/LessOrEqual.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/LessThan.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/NotEqual.java | 15 +++++++-------- .../java/com/alibaba/fluss/predicate/NotIn.java | 15 +++++++-------- .../predicate/NullFalseLeafBinaryFunction.java | 15 +++++++-------- .../java/com/alibaba/fluss/predicate/Or.java | 15 +++++++-------- .../predicate/PartitionPredicateVisitor.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/Predicate.java | 17 +++++++++-------- .../fluss/predicate/PredicateBuilder.java | 15 +++++++-------- .../fluss/predicate/PredicateVisitor.java | 15 +++++++-------- .../com/alibaba/fluss/predicate/StartsWith.java | 15 +++++++-------- .../fluss/predicate/PredicateBuilderTest.java | 15 +++++++-------- .../alibaba/fluss/predicate/PredicateTest.java | 15 +++++++-------- 29 files changed, 205 insertions(+), 232 deletions(-) diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java index d8b74b1a37..6793ad8306 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java index e2302af834..50380abf89 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java index f87dd1edd6..449598f2f9 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java index eab425a927..95c7a30f39 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java index ba397ca6ca..cd349be3c6 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java index 3105aab1ac..26b1817492 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java index 732b188e37..b9e179ddcd 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java index 7c2133be78..806f2de376 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java index 9198e24dee..0a915355fb 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java index e70ffe50a5..f92c84c9bb 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java index 24b1244ddb..6697750ce2 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java index 9defa44fd3..6249306b59 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java index 303d282b0a..d97c8cb433 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java index 5a2f7b5f13..a897c902e6 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java index ba9783ffa5..35ef76e919 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java index a1ce6721c1..e03710aa01 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java index f4b6b382c6..45dd3fda65 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java index 40554da7f6..ed1ee70c3a 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java index 4f8ec5bdc7..bca203a22b 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java index f20d6c5839..fa7cbfef3e 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java index ffeaa896db..da170834eb 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java index f3007c8aaf..b348a54781 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java index 3e399d4fce..6318aba9c8 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java index 55b6e3d2c0..e3d072ba51 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, @@ -36,6 +35,8 @@ public interface Predicate extends Serializable { /** + * Now only support test based on the specific input row. + * Todo: boolean test(long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts); * Test based on the specific input row. * * @return return true when hit, false when not hit. diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java index 83d5c5bc01..a90925204e 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java index d5b1e049ba..3e20e9c92a 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java index 984a4a5dcd..47e4ea4c78 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java index 1de02da0c8..acb193fe3e 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java index 471992d263..ef3132cd37 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, From 29d525a2a1c39dd7cdcba3151819ae7f006c3890 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Fri, 20 Jun 2025 17:20:25 +0800 Subject: [PATCH 08/22] [common]Introduce Predicate to do filter and partition push down. --- .../main/java/com/alibaba/fluss/predicate/Predicate.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java index e3d072ba51..7083a33d5b 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java @@ -35,9 +35,9 @@ public interface Predicate extends Serializable { /** - * Now only support test based on the specific input row. - * Todo: boolean test(long rowCount, InternalRow minValues, InternalRow maxValues, InternalArray nullCounts); - * Test based on the specific input row. + * Now only support test based on the specific input row. Todo: boolean test(long rowCount, + * InternalRow minValues, InternalRow maxValues, InternalArray nullCounts); Test based on the + * specific input row. * * @return return true when hit, false when not hit. */ From b9fad63087d61bed885098550e3635c13f8e344b Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:42:42 +0800 Subject: [PATCH 09/22] [flink]fix and add log --- .../fluss/flink/source/FlinkSource.java | 81 ++++------------ .../fluss/flink/source/FlinkTableSource.java | 12 +-- .../fluss/flink/source/FlussSource.java | 4 +- .../enumerator/FlinkSourceEnumerator.java | 67 +++++++------ .../flink/source/FlinkTableSourceITCase.java | 97 ++++++++++++------- 5 files changed, 124 insertions(+), 137 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java index 9687b5efc8..62bd92d3c0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java @@ -17,24 +17,22 @@ package org.apache.fluss.flink.source; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.source.deserializer.DeserializerInitContextImpl; -import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; -import org.apache.fluss.flink.source.emitter.FlinkRecordEmitter; -import org.apache.fluss.flink.source.enumerator.FlinkSourceEnumerator; -import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; -import org.apache.fluss.flink.source.metrics.FlinkSourceReaderMetrics; -import org.apache.fluss.flink.source.reader.FlinkSourceReader; -import org.apache.fluss.flink.source.reader.RecordAndPos; -import org.apache.fluss.flink.source.split.SourceSplitBase; -import org.apache.fluss.flink.source.split.SourceSplitSerializer; -import org.apache.fluss.flink.source.state.FlussSourceEnumeratorStateSerializer; -import org.apache.fluss.flink.source.state.SourceEnumeratorState; -import org.apache.fluss.flink.utils.PushdownUtils.FieldEqual; -import org.apache.fluss.lake.source.LakeSource; -import org.apache.fluss.lake.source.LakeSplit; -import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.types.RowType; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.flink.source.deserializer.DeserializerInitContextImpl; +import com.alibaba.fluss.flink.source.deserializer.FlussDeserializationSchema; +import com.alibaba.fluss.flink.source.emitter.FlinkRecordEmitter; +import com.alibaba.fluss.flink.source.enumerator.FlinkSourceEnumerator; +import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +import com.alibaba.fluss.flink.source.metrics.FlinkSourceReaderMetrics; +import com.alibaba.fluss.flink.source.reader.FlinkSourceReader; +import com.alibaba.fluss.flink.source.reader.RecordAndPos; +import com.alibaba.fluss.flink.source.split.SourceSplitBase; +import com.alibaba.fluss.flink.source.split.SourceSplitSerializer; +import com.alibaba.fluss.flink.source.state.FlussSourceEnumeratorStateSerializer; +import com.alibaba.fluss.flink.source.state.SourceEnumeratorState; +import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.predicate.Predicate; +import com.alibaba.fluss.types.RowType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -50,10 +48,6 @@ import javax.annotation.Nullable; -import java.util.List; - -import static org.apache.fluss.utils.Preconditions.checkNotNull; - /** Flink source for Fluss. */ public class FlinkSource implements Source, ResultTypeQueryable { @@ -70,9 +64,7 @@ public class FlinkSource private final boolean streaming; private final FlussDeserializationSchema deserializationSchema; - private final List partitionFilters; - - private final @Nullable LakeSource lakeSource; + private Predicate predicate; public FlinkSource( Configuration flussConf, @@ -85,35 +77,7 @@ public FlinkSource( long scanPartitionDiscoveryIntervalMs, FlussDeserializationSchema deserializationSchema, boolean streaming, - List partitionFilters) { - this( - flussConf, - tablePath, - hasPrimaryKey, - isPartitioned, - sourceOutputType, - projectedFields, - offsetsInitializer, - scanPartitionDiscoveryIntervalMs, - deserializationSchema, - streaming, - partitionFilters, - null); - } - - public FlinkSource( - Configuration flussConf, - TablePath tablePath, - boolean hasPrimaryKey, - boolean isPartitioned, - RowType sourceOutputType, - @Nullable int[] projectedFields, - OffsetsInitializer offsetsInitializer, - long scanPartitionDiscoveryIntervalMs, - FlussDeserializationSchema deserializationSchema, - boolean streaming, - List partitionFilters, - LakeSource lakeSource) { + Predicate predicate) { this.flussConf = flussConf; this.tablePath = tablePath; this.hasPrimaryKey = hasPrimaryKey; @@ -124,8 +88,7 @@ public FlinkSource( this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.deserializationSchema = deserializationSchema; this.streaming = streaming; - this.partitionFilters = checkNotNull(partitionFilters); - this.lakeSource = lakeSource; + this.predicate = predicate; } @Override @@ -145,8 +108,7 @@ public SplitEnumerator createEnumerator( offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters, - lakeSource); + predicate); } @Override @@ -165,8 +127,7 @@ public SplitEnumerator restoreEnumerator offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters, - lakeSource); + predicate); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 895169ec4c..11ad78a3c8 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -82,13 +82,7 @@ import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.fluss.flink.utils.LakeSourceUtils.createLakeSource; -import static org.apache.fluss.flink.utils.PushdownUtils.ValueConversion.FLINK_INTERNAL_VALUE; -import static org.apache.fluss.flink.utils.PushdownUtils.ValueConversion.FLUSS_INTERNAL_VALUE; -import static org.apache.fluss.flink.utils.PushdownUtils.extractFieldEquals; -import static org.apache.fluss.metadata.TableDescriptor.TIMESTAMP_COLUMN_NAME; -import static org.apache.fluss.utils.Preconditions.checkNotNull; -import static org.apache.fluss.utils.Preconditions.checkState; +import static com.alibaba.fluss.utils.Preconditions.checkNotNull; /** Flink table source to scan Fluss data. */ public class FlinkTableSource @@ -426,8 +420,6 @@ public DynamicTableSource copy() { source.projectedFields = projectedFields; source.singleRowFilter = singleRowFilter; source.modificationScanType = modificationScanType; - source.partitionFilters = partitionFilters; - source.lakeSource = lakeSource; return source; } @@ -473,7 +465,7 @@ && hasPrimaryKey() primaryKeyTypes, acceptedFilters, remainingFilters, - ValueConversion.FLINK_INTERNAL_VALUE); + PushdownUtils.ValueConversion.FLINK_INTERNAL_VALUE); int[] keyRowProjection = getKeyRowProjection(); HashSet visitedPkFields = new HashSet<>(); GenericRowData lookupRow = new GenericRowData(primaryKeyIndexes.length); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java index 428204d83f..4277418349 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlussSource.java @@ -26,8 +26,6 @@ import javax.annotation.Nullable; -import java.util.Collections; - /** * A Flink DataStream source implementation for reading data from Fluss tables. * @@ -82,7 +80,7 @@ public class FlussSource extends FlinkSource { scanPartitionDiscoveryIntervalMs, deserializationSchema, streaming, - Collections.emptyList()); + null); } /** diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index 6e28eeab8c..3e532d0fec 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -23,18 +23,18 @@ import com.alibaba.fluss.client.metadata.KvSnapshots; import com.alibaba.fluss.config.ConfigOptions; import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.connector.flink.lakehouse.LakeSplitGenerator; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; -import com.alibaba.fluss.connector.flink.source.event.PartitionBucketsUnsubscribedEvent; -import com.alibaba.fluss.connector.flink.source.event.PartitionsRemovedEvent; -import com.alibaba.fluss.connector.flink.source.split.HybridSnapshotLogSplit; -import com.alibaba.fluss.connector.flink.source.split.LogSplit; -import com.alibaba.fluss.connector.flink.source.split.SourceSplitBase; -import com.alibaba.fluss.connector.flink.source.state.SourceEnumeratorState; +import com.alibaba.fluss.flink.lakehouse.LakeSplitGenerator; +import com.alibaba.fluss.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; +import com.alibaba.fluss.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; +import com.alibaba.fluss.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; +import com.alibaba.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; +import com.alibaba.fluss.flink.source.event.PartitionsRemovedEvent; +import com.alibaba.fluss.flink.source.split.HybridSnapshotLogSplit; +import com.alibaba.fluss.flink.source.split.LogSplit; +import com.alibaba.fluss.flink.source.split.SourceSplitBase; +import com.alibaba.fluss.flink.source.state.SourceEnumeratorState; import com.alibaba.fluss.metadata.PartitionInfo; import com.alibaba.fluss.metadata.TableBucket; import com.alibaba.fluss.metadata.TableInfo; @@ -211,7 +211,6 @@ public FlinkSourceEnumerator( : new LinkedList<>(pendingHybridLakeFlussSplits); this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.streaming = streaming; - this.partitionFilters = checkNotNull(partitionFilters); this.stoppingOffsetsInitializer = streaming ? new NoStoppingOffsetsInitializer() : OffsetsInitializer.latest(); this.predicate = predicate; @@ -336,24 +335,8 @@ private Set listPartitions() { try { List partitionInfos = flussAdmin.listPartitionInfos(tablePath).get(); - if (predicate == null) { - return new HashSet<>(partitionInfos); - } else { - Set filteredPartitionInfos = - partitionInfos.stream() - .filter( - partitionInfo -> - predicate.test( - convertPartitionInfoToInternalRow( - partitionInfo))) - .collect(Collectors.toSet()); - LOG.info( - "Filtered partitions {} for table {} with predicate: {}", - filteredPartitionInfos, - tablePath, - predicate); - return filteredPartitionInfos; - } + partitionInfos = applyPartitionFilter(partitionInfos); + return new HashSet<>(partitionInfos); } catch (Exception e) { throw new FlinkRuntimeException( String.format("Failed to list partitions for %s", tablePath), @@ -361,6 +344,28 @@ private Set listPartitions() { } } + /** Apply partition filter. */ + private List applyPartitionFilter(List partitionInfos) { + if (predicate == null) { + return partitionInfos; + } else { + List filteredPartitionInfos = + partitionInfos.stream() + .filter( + partitionInfo -> + predicate.test( + convertPartitionInfoToInternalRow( + partitionInfo))) + .collect(Collectors.toList()); + LOG.info( + "Filtered partitions {} for table {} with predicate: {}", + filteredPartitionInfos, + tablePath, + predicate); + return filteredPartitionInfos; + } + } + private InternalRow convertPartitionInfoToInternalRow(PartitionInfo partitionInfo) { return GenericRow.of(BinaryString.fromString(partitionInfo.getPartitionName())); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 3cc416618b..68236ed4da 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -708,39 +708,6 @@ void testReadTimestampGreaterThanMaxTimestamp() throws Exception { } } - @Test - void testStreamingReadPartitionPushDown() throws Exception { - - tEnv.executeSql( - "create table partitioned_table" - + " (a int not null, b varchar, c string, primary key (a, c) NOT ENFORCED) partitioned by (c) " - + "with ('table.auto-partition.enabled' = 'true', 'table.auto-partition.time-unit' = 'year')"); - TablePath tablePath = TablePath.of(DEFAULT_DB, "partitioned_table"); - - // write data into partitions and wait snapshot is done - Map partitionNameById = - waitUntilPartitions(FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath); - List expectedRowValues = - writeRowsToPartition(tablePath, partitionNameById.values()).stream() - .filter(s -> s.contains("2025")) - .collect(Collectors.toList()); - waitUtilAllBucketFinishSnapshot(admin, tablePath, partitionNameById.values()); - - org.apache.flink.util.CloseableIterator rowIter = - tEnv.executeSql("select * from partitioned_table where c in( '2000','2001','2025')") - .collect(); - - assertResultsIgnoreOrder(rowIter, expectedRowValues, false); - - // then create some new partitions, and write rows to the new partitions - List newPartitions = Arrays.asList("2000", "2001"); - FlinkTestBase.createPartitions( - FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath, newPartitions); - // write data to the new partitions - expectedRowValues = writeRowsToPartition(tablePath, newPartitions); - assertResultsIgnoreOrder(rowIter, expectedRowValues, true); - } - // ------------------------------------------------------------------------------------- // Fluss look source tests // ------------------------------------------------------------------------------------- @@ -1184,6 +1151,70 @@ private List writeRowsToTwoPartition(TablePath tablePath, Collection partitionNameById = + // waitUntilPartitions(FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), + // tablePath); + // List expectedRowValues = + // writeRowsToPartition(tablePath, partitionNameById.values()).stream() + // .filter(s -> s.contains("2025")) + // .collect(Collectors.toList()); + // waitUtilAllBucketFinishSnapshot(admin, tablePath, partitionNameById.values()); + // + // org.apache.flink.util.CloseableIterator rowIter = + // tEnv.executeSql("select * from partitioned_table where c in( + // '2000','2001','2025')") + // .collect(); + // + // assertResultsIgnoreOrder(rowIter, expectedRowValues, false); + // + // // then create some new partitions, and write rows to the new partitions + // List newPartitions = Arrays.asList("2000", "2001"); + // FlinkTestBase.createPartitions( + // FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath, newPartitions); + // // write data to the new partitions + // expectedRowValues = writeRowsToPartition(tablePath, newPartitions); + // assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + + tEnv.executeSql( + "create table partitioned_table_in" + + " (a int not null, b varchar, c string, primary key (a, c) NOT ENFORCED) partitioned by (c) "); + TablePath tablePath = TablePath.of(DEFAULT_DB, "partitioned_table_in"); + tEnv.executeSql("alter table partitioned_table_in add partition (c=2025)"); + tEnv.executeSql("alter table partitioned_table_in add partition (c=2026)"); + tEnv.executeSql("alter table partitioned_table_in add partition (c=2027)"); + + List expectedRowValues = + writeRowsToPartition(conn, tablePath, Arrays.asList("2025", "2026", "2027")) + .stream() + .filter(s -> s.contains("2025") || s.contains("2026")) + .collect(Collectors.toList()); + waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); + + String plan = + tEnv.explainSql("select * from partitioned_table_in where c in ('2025','2026')"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_in, filter=[OR(=(c, _UTF-16LE'2025'), =(c, _UTF-16LE'2026'))]]], fields=[a, b, c])"); + + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql("select * from partitioned_table_in where c in ('2025','2026')") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + private enum Caching { ENABLE_CACHE, DISABLE_CACHE From d5e78865afd05dcd0259433f923ad192f6c7493d Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 23 Jun 2025 10:07:28 +0800 Subject: [PATCH 10/22] [flink]Support partition pushdown in Flink connector --- .../flink/source/FlinkTableSourceITCase.java | 33 ------------------- 1 file changed, 33 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 68236ed4da..8a9939d486 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1154,39 +1154,6 @@ private List writeRowsToTwoPartition(TablePath tablePath, Collection partitionNameById = - // waitUntilPartitions(FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), - // tablePath); - // List expectedRowValues = - // writeRowsToPartition(tablePath, partitionNameById.values()).stream() - // .filter(s -> s.contains("2025")) - // .collect(Collectors.toList()); - // waitUtilAllBucketFinishSnapshot(admin, tablePath, partitionNameById.values()); - // - // org.apache.flink.util.CloseableIterator rowIter = - // tEnv.executeSql("select * from partitioned_table where c in( - // '2000','2001','2025')") - // .collect(); - // - // assertResultsIgnoreOrder(rowIter, expectedRowValues, false); - // - // // then create some new partitions, and write rows to the new partitions - // List newPartitions = Arrays.asList("2000", "2001"); - // FlinkTestBase.createPartitions( - // FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(), tablePath, newPartitions); - // // write data to the new partitions - // expectedRowValues = writeRowsToPartition(tablePath, newPartitions); - // assertResultsIgnoreOrder(rowIter, expectedRowValues, true); - tEnv.executeSql( "create table partitioned_table_in" + " (a int not null, b varchar, c string, primary key (a, c) NOT ENFORCED) partitioned by (c) "); From b0180f4a23cc09b65e9f47886324e6f51f762ae2 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 23 Jun 2025 14:22:41 +0800 Subject: [PATCH 11/22] [flink]some fix --- .../fluss/flink/source/FlinkTableSource.java | 1 + .../flink/source/FlinkTableSourceITCase.java | 70 +++++++++++++++++++ 2 files changed, 71 insertions(+) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 11ad78a3c8..e871fa0660 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -420,6 +420,7 @@ public DynamicTableSource copy() { source.projectedFields = projectedFields; source.singleRowFilter = singleRowFilter; source.modificationScanType = modificationScanType; + source.predicate = predicate; return source; } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 8a9939d486..faf5cb7a10 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1180,6 +1180,76 @@ void testStreamingReadPartitionPushDownWithInExpr() throws Exception { .collect(); assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + + plan = tEnv.explainSql("select * from partitioned_table_in where c ='2025' or c ='2026'"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_in, filter=[OR(=(c, _UTF-16LE'2025':VARCHAR(2147483647) CHARACTER SET \"UTF-16LE\"), =(c, _UTF-16LE'2026':VARCHAR(2147483647) CHARACTER SET \"UTF-16LE\"))]]], fields=[a, b, c])"); + + rowIter = + tEnv.executeSql("select * from partitioned_table_in where c ='2025' or c ='2026'") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + + @Test + void testStreamingReadWithCombinedFiltersAndInExpr() throws Exception { + tEnv.executeSql( + "create table combined_filters_table_in" + + " (a int not null, b varchar, c string, d int, primary key (a, c) NOT ENFORCED) partitioned by (c) "); + TablePath tablePath = TablePath.of(DEFAULT_DB, "combined_filters_table_in"); + tEnv.executeSql("alter table combined_filters_table_in add partition (c=2025)"); + tEnv.executeSql("alter table combined_filters_table_in add partition (c=2026)"); + tEnv.executeSql("alter table combined_filters_table_in add partition (c=2027)"); + + List rows = new ArrayList<>(); + List expectedRowValues = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + rows.add(row(i, "v" + i, "2025", i * 100)); + if (i % 2 == 0) { + expectedRowValues.add(String.format("+I[%d, 2025, %d]", i, i * 100)); + } + } + for (int i = 0; i < 10; i++) { + rows.add(row(i, "v" + i, "2026", i * 100)); + if (i % 2 == 0) { + expectedRowValues.add(String.format("+I[%d, 2026, %d]", i, i * 100)); + } + } + writeRows(conn, tablePath, rows, false); + + for (int i = 0; i < 10; i++) { + rows.add(row(i, "v" + i, "2027", i * 100)); + } + + writeRows(conn, tablePath, rows, false); + waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); + + String plan = + tEnv.explainSql( + "select a,c,d from combined_filters_table_in where c in ('2025','2026') and d % 200 = 0"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, combined_filters_table_in, filter=[OR(=(c, _UTF-16LE'2025'), =(c, _UTF-16LE'2026'))], project=[a, c, d]]], fields=[a, c, d])"); + + // test column filter、partition filter and flink runtime filter + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql( + "select a,c,d from combined_filters_table_in where c in ('2025','2026') " + + "and d % 200 = 0") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + + rowIter = + tEnv.executeSql( + "select a,c,d from combined_filters_table_in where (c ='2025' or c ='2026') " + + "and d % 200 = 0") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); } private enum Caching { From 2fde16a5cf8c9033fc00c34fdd77875ee1268986 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 25 Jun 2025 16:59:08 +0800 Subject: [PATCH 12/22] [flink]some fix --- .../flink/source/enumerator/FlinkSourceEnumerator.java | 8 +++++++- .../apache/fluss/flink/source/FlinkTableSourceITCase.java | 4 ++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index 3e532d0fec..bd2d6c5a7a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -367,7 +367,13 @@ private List applyPartitionFilter(List partitionIn } private InternalRow convertPartitionInfoToInternalRow(PartitionInfo partitionInfo) { - return GenericRow.of(BinaryString.fromString(partitionInfo.getPartitionName())); + List partitionValues = + partitionInfo.getResolvedPartitionSpec().getPartitionValues(); + GenericRow genericRow = new GenericRow(partitionValues.size()); + for (int i = 0; i < partitionValues.size(); i++) { + genericRow.setField(i, BinaryString.fromString(partitionValues.get(i))); + } + return genericRow; } /** Init the splits for Fluss. */ diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index faf5cb7a10..6b9defea65 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1024,8 +1024,8 @@ void testStreamingReadMultiPartitionPushDown() throws Exception { .stream() .filter(s -> s.contains("2025")) .collect(Collectors.toList()); - waitUntilAllBucketFinishSnapshot( - admin, tablePath, Arrays.asList("2025$1", "2025$2", "2025$2")); + waitUtilAllBucketFinishSnapshot( + admin, tablePath, Arrays.asList("2025$1", "2025$2", "2026$1")); String plan = tEnv.explainSql("select * from multi_partitioned_table where c ='2025'"); assertThat(plan) From 0dd561d8a6e21bfb656d1aa0f932ff8e27d1b86e Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 25 Jun 2025 17:11:48 +0800 Subject: [PATCH 13/22] [flink]some fix --- .../java/org/apache/fluss/flink/source/FlinkTableSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index e871fa0660..72db4bcfc2 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -520,7 +520,7 @@ && hasPrimaryKey() return Result.of(acceptedFilters, remainingFilters); } - return Result.of(acceptedFilters, remainingFilters); + return Result.of(Collections.emptyList(), filters); } @Override From 4c7e199e1c69c047373cf07c5eb433de01ba9d54 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 25 Jun 2025 17:26:46 +0800 Subject: [PATCH 14/22] [flink]some fix --- .../fluss/flink/source/enumerator/FlinkSourceEnumerator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index bd2d6c5a7a..b620d1d9f7 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -187,8 +187,8 @@ public FlinkSourceEnumerator( public FlinkSourceEnumerator( TablePath tablePath, Configuration flussConf, - boolean isPartitioned, boolean hasPrimaryKey, + boolean isPartitioned, SplitEnumeratorContext context, Set assignedTableBuckets, Map assignedPartitions, From 85c6408410ebea05b574ceb09f9eee82e9504c80 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 25 Jun 2025 17:52:38 +0800 Subject: [PATCH 15/22] [flink]some fix --- .../java/org/apache/fluss/flink/source/FlinkTableSource.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 72db4bcfc2..d8fcef2704 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -480,7 +480,8 @@ && hasPrimaryKey() } singleRowFilter = lookupRow; return Result.of(acceptedFilters, remainingFilters); - } else if (isPartitioned()) { + } else if (isPartitioned() + && !RowLevelModificationType.UPDATE.equals(modificationScanType)) { // apply partition filter pushdown List converted = new ArrayList<>(); From 549da144e50c1e10e7d79435115b619b9f68abfa Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Mon, 30 Jun 2025 11:35:41 +0800 Subject: [PATCH 16/22] [flink]some fix --- .../java/org/apache/fluss/flink/source/FlinkTableSource.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index d8fcef2704..2530bbf377 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -493,6 +493,9 @@ && hasPrimaryKey() PredicateVisitor partitionPredicateVisitor = new PartitionPredicateVisitor(partitionKeys); + + // TODO after https://github.com/alibaba/fluss/pull/979 + // replace string type with the real type LogicalType[] partitionKeyTypes = partitionKeys.stream() .map(key -> VarCharType.STRING_TYPE) From 6a05045fefb372ad17becd029347c28db2bc6a15 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:44:03 +0800 Subject: [PATCH 17/22] [flink]fix and add log --- .../com/alibaba/fluss/predicate/Contains.java | 5 +- .../com/alibaba/fluss/predicate/EndsWith.java | 5 +- .../alibaba/fluss/predicate/StartsWith.java | 11 +- .../fluss/predicate/PredicateTest.java | 25 ++++- .../fluss/flink/row/FlinkAsFlussRow.java | 21 +++- .../fluss/flink/source/FlinkSource.java | 10 +- .../fluss/flink/source/FlinkTableSource.java | 8 +- .../flink/source/FlinkTableSourceITCase.java | 105 +++++++++++++++++- 8 files changed, 159 insertions(+), 31 deletions(-) diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java index 95c7a30f39..cfdaf0a06e 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java @@ -17,7 +17,6 @@ package com.alibaba.fluss.predicate; -import com.alibaba.fluss.row.BinaryString; import com.alibaba.fluss.types.DataType; import java.util.List; @@ -36,8 +35,8 @@ private Contains() {} @Override public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.contains((BinaryString) patternLiteral); + String fieldString = field.toString(); + return fieldString.contains((String) patternLiteral); } @Override diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java index cd349be3c6..a6af4c2e1d 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java @@ -17,7 +17,6 @@ package com.alibaba.fluss.predicate; -import com.alibaba.fluss.row.BinaryString; import com.alibaba.fluss.types.DataType; import java.util.List; @@ -39,8 +38,8 @@ private EndsWith() {} @Override public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.endsWith((BinaryString) patternLiteral); + String fieldString = field.toString(); + return fieldString.endsWith((String) patternLiteral); } @Override diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java index 47e4ea4c78..e3baa41fa9 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java @@ -17,7 +17,6 @@ package com.alibaba.fluss.predicate; -import com.alibaba.fluss.row.BinaryString; import com.alibaba.fluss.types.DataType; import java.util.List; @@ -39,8 +38,8 @@ private StartsWith() {} @Override public boolean test(DataType type, Object field, Object patternLiteral) { - BinaryString fieldString = (BinaryString) field; - return fieldString.startsWith((BinaryString) patternLiteral); + String fieldString = field.toString(); + return fieldString.startsWith((String) patternLiteral); } @Override @@ -51,9 +50,9 @@ public boolean test( Object max, Long nullCount, Object patternLiteral) { - BinaryString minStr = (BinaryString) min; - BinaryString maxStr = (BinaryString) max; - BinaryString pattern = (BinaryString) patternLiteral; + String minStr = min.toString(); + String maxStr = max.toString(); + String pattern = patternLiteral.toString(); return (minStr.startsWith(pattern) || minStr.compareTo(pattern) <= 0) && (maxStr.startsWith(pattern) || maxStr.compareTo(pattern) >= 0); } diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java index ef3132cd37..427fec8a82 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java @@ -237,15 +237,32 @@ public void testNotInNull() { @Test public void testEndsWith() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); - Predicate predicate = builder.endsWith(0, fromString("bcc")); + Predicate predicate = builder.endsWith(0, ("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); } + @Test + public void testStartWith() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); + Predicate predicate = builder.startsWith(0, ("aab")); + GenericRow row = GenericRow.of(fromString("aabbcc")); + + assertThat(predicate.test(row)).isEqualTo(true); + } + + @Test + public void testContainsWith() { + PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); + Predicate predicate = builder.contains(0, ("def")); + GenericRow row1 = GenericRow.of(fromString("aabbdefcc")); + GenericRow row2 = GenericRow.of(fromString("aabbdcefcc")); + + assertThat(predicate.test(row1)).isEqualTo(true); + assertThat(predicate.test(row2)).isEqualTo(false); + } + @Test public void testLargeIn() { PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java index 4b48a4ef45..fbe74c49d7 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java @@ -17,15 +17,18 @@ package org.apache.fluss.flink.row; -import org.apache.fluss.row.BinaryString; -import org.apache.fluss.row.Decimal; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.row.TimestampLtz; -import org.apache.fluss.row.TimestampNtz; +import com.alibaba.fluss.flink.utils.FlinkConversions; +import com.alibaba.fluss.row.BinaryString; +import com.alibaba.fluss.row.Decimal; +import com.alibaba.fluss.row.InternalRow; +import com.alibaba.fluss.row.TimestampLtz; +import com.alibaba.fluss.row.TimestampNtz; import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; /** Wraps a Flink {@link RowData} as a Fluss {@link InternalRow}. */ public class FlinkAsFlussRow implements InternalRow { @@ -132,4 +135,12 @@ public byte[] getBinary(int pos, int length) { public byte[] getBytes(int pos) { return flinkRow.getBinary(pos); } + + public static Object fromFlinkObject(Object o, DataType type) { + if (o == null) { + return null; + } + return InternalRow.createFieldGetter(FlinkConversions.toFlussType(type), 0) + .getFieldOrNull((new FlinkAsFlussRow()).replace(GenericRowData.of(o))); + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java index 62bd92d3c0..e72325cfe0 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java @@ -64,7 +64,7 @@ public class FlinkSource private final boolean streaming; private final FlussDeserializationSchema deserializationSchema; - private Predicate predicate; + private Predicate partitionFilters; public FlinkSource( Configuration flussConf, @@ -77,7 +77,7 @@ public FlinkSource( long scanPartitionDiscoveryIntervalMs, FlussDeserializationSchema deserializationSchema, boolean streaming, - Predicate predicate) { + Predicate partitionFilters) { this.flussConf = flussConf; this.tablePath = tablePath; this.hasPrimaryKey = hasPrimaryKey; @@ -88,7 +88,7 @@ public FlinkSource( this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.deserializationSchema = deserializationSchema; this.streaming = streaming; - this.predicate = predicate; + this.partitionFilters = partitionFilters; } @Override @@ -108,7 +108,7 @@ public SplitEnumerator createEnumerator( offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - predicate); + partitionFilters); } @Override @@ -127,7 +127,7 @@ public SplitEnumerator restoreEnumerator offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - predicate); + partitionFilters); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 2530bbf377..627c8f9a97 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -134,7 +134,7 @@ public class FlinkTableSource private long limit = -1; - @Nullable protected Predicate predicate; + @Nullable protected Predicate partitionFilters; public FlinkTableSource( TablePath tablePath, @@ -295,7 +295,7 @@ public boolean isBounded() { offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - predicate); + partitionFilters); if (!streaming) { // return a bounded source provide to make planner happy, @@ -420,7 +420,7 @@ public DynamicTableSource copy() { source.projectedFields = projectedFields; source.singleRowFilter = singleRowFilter; source.modificationScanType = modificationScanType; - source.predicate = predicate; + source.partitionFilters = partitionFilters; return source; } @@ -520,7 +520,7 @@ && hasPrimaryKey() converted.add(p); } } - predicate = converted.isEmpty() ? null : PredicateBuilder.and(converted); + partitionFilters = converted.isEmpty() ? null : PredicateBuilder.and(converted); return Result.of(acceptedFilters, remainingFilters); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 6b9defea65..b428713eee 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1067,7 +1067,7 @@ void testStreamingReadMultiPartitionPushDown() throws Exception { } @Test - void testStreamingReadWithCombinedFilters() throws Exception { + void testStreamingReadWithCombinedFilters1() throws Exception { tEnv.executeSql( "create table combined_filters_table" + " (a int not null, b varchar, c string, d int, primary key (a, c) NOT ENFORCED) partitioned by (c) "); @@ -1111,6 +1111,51 @@ void testStreamingReadWithCombinedFilters() throws Exception { assertResultsIgnoreOrder(rowIter, expectedRowValues, true); } + @Test + void testStreamingReadWithCombinedFilters2() throws Exception { + tEnv.executeSql( + "create table combined_filters_table" + + " (a int not null, b varchar, c string, d int, primary key (a, c) NOT ENFORCED) partitioned by (c) "); + TablePath tablePath = TablePath.of(DEFAULT_DB, "combined_filters_table"); + tEnv.executeSql("alter table combined_filters_table add partition (c=2025)"); + tEnv.executeSql("alter table combined_filters_table add partition (c=2026)"); + + List rows = new ArrayList<>(); + List expectedRowValues = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + rows.add(row(i, "v" + i, "2025", i * 100)); + if (i == 2) { + expectedRowValues.add(String.format("+I[%d, 2025, %d]", i, i * 100)); + } + } + writeRows(conn, tablePath, rows, false); + + for (int i = 0; i < 10; i++) { + rows.add(row(i, "v" + i, "2026", i * 100)); + } + + writeRows(conn, tablePath, rows, false); + waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026")); + + String plan = + tEnv.explainSql( + "select a,c,d from combined_filters_table where c ='2025' and d = 200"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, combined_filters_table, " + + "filter=[=(c, _UTF-16LE'2025':VARCHAR(2147483647) CHARACTER SET \"UTF-16LE\")], " + + "project=[a, d]]], fields=[a, d])"); + + // test column filter、partition filter and flink runtime filter + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql( + "select a,c,d from combined_filters_table where c ='2025' and d = 200") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + @Test void testNonPartitionPushDown() throws Exception { tEnv.executeSql( @@ -1252,6 +1297,64 @@ void testStreamingReadWithCombinedFiltersAndInExpr() throws Exception { assertResultsIgnoreOrder(rowIter, expectedRowValues, true); } + @Test + void testStreamingReadPartitionPushDownWithLikeExpr() throws Exception { + + tEnv.executeSql( + "create table partitioned_table_like" + + " (a int not null, b varchar, c string, primary key (a, c) NOT ENFORCED) partitioned by (c) "); + TablePath tablePath = TablePath.of(DEFAULT_DB, "partitioned_table_like"); + tEnv.executeSql("alter table partitioned_table_like add partition (c=2025)"); + tEnv.executeSql("alter table partitioned_table_like add partition (c=2026)"); + tEnv.executeSql("alter table partitioned_table_like add partition (c=3026)"); + + List allData = + writeRowsToPartition(conn, tablePath, Arrays.asList("2025", "2026", "3026")); + List expectedRowValues = + allData.stream() + .filter(s -> s.contains("2025") || s.contains("2026")) + .collect(Collectors.toList()); + waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "3026")); + + String plan = tEnv.explainSql("select * from partitioned_table_like where c like '202%'"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_like, filter=[LIKE(c, _UTF-16LE'202%')]]], fields=[a, b, c])"); + + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql("select * from partitioned_table_like where c like '202%'") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + expectedRowValues = + allData.stream() + .filter(s -> s.contains("2026") || s.contains("3026")) + .collect(Collectors.toList()); + plan = tEnv.explainSql("select * from partitioned_table_like where c like '%026'"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_like, filter=[LIKE(c, _UTF-16LE'%026')]]], fields=[a, b, c])"); + + rowIter = + tEnv.executeSql("select * from partitioned_table_like where c like '%026'") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + + expectedRowValues = + allData.stream().filter(s -> s.contains("3026")).collect(Collectors.toList()); + plan = tEnv.explainSql("select * from partitioned_table_like where c like '%3026%'"); + assertThat(plan) + .contains( + "TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_like, filter=[LIKE(c, _UTF-16LE'%3026%')]]], fields=[a, b, c])"); + + rowIter = + tEnv.executeSql("select * from partitioned_table_like where c like '%3026%'") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + private enum Caching { ENABLE_CACHE, DISABLE_CACHE From 3a5077b14aa3c13845986ac1b49002d1c959ebfb Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:44:12 +0800 Subject: [PATCH 18/22] [flink]fix and add log --- .../fluss/predicate/PredicateBuilder.java | 9 ++++ .../predicate/UnsupportedExpression.java | 29 ++++++++++++ .../flink/source/FlinkTableSourceITCase.java | 46 ++++++++++++++++++- 3 files changed, 82 insertions(+), 2 deletions(-) create mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java index a90925204e..975954c2a7 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java @@ -121,18 +121,21 @@ public Predicate contains(int idx, Object patternLiteral) { } public Predicate leaf(NullFalseLeafBinaryFunction function, int idx, Object literal) { + validateIndex(idx); DataField field = rowType.getFields().get(idx); return new LeafPredicate( function, field.getType(), idx, field.getName(), singletonList(literal)); } public Predicate leaf(LeafUnaryFunction function, int idx) { + validateIndex(idx); DataField field = rowType.getFields().get(idx); return new LeafPredicate( function, field.getType(), idx, field.getName(), Collections.emptyList()); } public Predicate in(int idx, List literals) { + validateIndex(idx); // In the IN predicate, 20 literals are critical for performance. // If there are more than 20 literals, the performance will decrease. if (literals.size() > 20) { @@ -233,6 +236,12 @@ private static void splitCompound( } } + private void validateIndex(int idx) { + if (idx < 0 || idx >= rowType.getFieldCount()) { + throw new UnsupportedExpression("idx is not valid"); + } + } + public static Object convertJavaObject(DataType literalType, Object o) { if (o == null) { return null; diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java new file mode 100644 index 0000000000..74b50f2f4e --- /dev/null +++ b/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java @@ -0,0 +1,29 @@ +/* + * 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 com.alibaba.fluss.predicate; + +/** Encounter an unsupported expression, the caller can choose to ignore this filter branch. */ +public class UnsupportedExpression extends RuntimeException { + public UnsupportedExpression(String message) { + super(message); + } + + public UnsupportedExpression() { + super(); + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index b428713eee..2da11fa5d9 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1186,8 +1186,9 @@ private List writeRowsToTwoPartition(TablePath tablePath, Collection allData = + writeRowsToTwoPartition( + tablePath, Arrays.asList("c=2025,d=1", "c=2025,d=2", "c=2026,d=1")); + List expectedRowValues = + allData.stream() + .filter(s -> s.contains("v3") && !s.contains("2025, 2")) + .collect(Collectors.toList()); + waitUtilAllBucketFinishSnapshot( + admin, tablePath, Arrays.asList("2025$1", "2025$2", "2026$1")); + + String plan = + tEnv.explainSql( + "select * from partitioned_table_complex where a = 3\n" + + " and (c in ('2026') or d like '%1%') " + + " and b like '%v3%'"); + assertThat(plan) + .contains( + "Calc(select=[3 AS a, b, c, d], where=[((a = 3) AND LIKE(b, '%v3%'))])\n" + + "+- TableSourceScan(table=[[testcatalog, defaultdb, partitioned_table_complex, filter=[OR(=(c, _UTF-16LE'2026'), LIKE(d, _UTF-16LE'%1%'))]]], fields=[a, b, c, d])"); + + org.apache.flink.util.CloseableIterator rowIter = + tEnv.executeSql( + "select * from partitioned_table_complex where a = 3\n" + + " and (c in ('2026') or d like '%1%') " + + " and b like '%v3%'") + .collect(); + + assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + } + private enum Caching { ENABLE_CACHE, DISABLE_CACHE From 42551882720813b3103d6f77176e58492d45512e Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Fri, 25 Jul 2025 10:41:46 +0800 Subject: [PATCH 19/22] [flink]some fix --- .../flink/source/FlinkTableSourceITCase.java | 44 +++++-------------- 1 file changed, 10 insertions(+), 34 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 2da11fa5d9..5f44640181 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1076,12 +1076,16 @@ void testStreamingReadWithCombinedFilters1() throws Exception { tEnv.executeSql("alter table combined_filters_table add partition (c=2026)"); List rows = new ArrayList<>(); - List expectedRowValues = new ArrayList<>(); + List expectedRowValues1 = new ArrayList<>(); + List expectedRowValues2 = new ArrayList<>(); for (int i = 0; i < 10; i++) { rows.add(row(i, "v" + i, "2025", i * 100)); if (i % 2 == 0) { - expectedRowValues.add(String.format("+I[%d, 2025, %d]", i, i * 100)); + expectedRowValues1.add(String.format("+I[%d, 2025, %d]", i, i * 100)); + } + if (i == 2) { + expectedRowValues2.add(String.format("+I[%d, 2025, %d]", i, i * 100)); } } writeRows(conn, tablePath, rows, false); @@ -1108,37 +1112,9 @@ void testStreamingReadWithCombinedFilters1() throws Exception { "select a,c,d from combined_filters_table where c ='2025' and d % 200 = 0") .collect(); - assertResultsIgnoreOrder(rowIter, expectedRowValues, true); - } - - @Test - void testStreamingReadWithCombinedFilters2() throws Exception { - tEnv.executeSql( - "create table combined_filters_table" - + " (a int not null, b varchar, c string, d int, primary key (a, c) NOT ENFORCED) partitioned by (c) "); - TablePath tablePath = TablePath.of(DEFAULT_DB, "combined_filters_table"); - tEnv.executeSql("alter table combined_filters_table add partition (c=2025)"); - tEnv.executeSql("alter table combined_filters_table add partition (c=2026)"); - - List rows = new ArrayList<>(); - List expectedRowValues = new ArrayList<>(); + assertResultsIgnoreOrder(rowIter, expectedRowValues1, true); - for (int i = 0; i < 10; i++) { - rows.add(row(i, "v" + i, "2025", i * 100)); - if (i == 2) { - expectedRowValues.add(String.format("+I[%d, 2025, %d]", i, i * 100)); - } - } - writeRows(conn, tablePath, rows, false); - - for (int i = 0; i < 10; i++) { - rows.add(row(i, "v" + i, "2026", i * 100)); - } - - writeRows(conn, tablePath, rows, false); - waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026")); - - String plan = + plan = tEnv.explainSql( "select a,c,d from combined_filters_table where c ='2025' and d = 200"); assertThat(plan) @@ -1148,12 +1124,12 @@ void testStreamingReadWithCombinedFilters2() throws Exception { + "project=[a, d]]], fields=[a, d])"); // test column filter、partition filter and flink runtime filter - org.apache.flink.util.CloseableIterator rowIter = + rowIter = tEnv.executeSql( "select a,c,d from combined_filters_table where c ='2025' and d = 200") .collect(); - assertResultsIgnoreOrder(rowIter, expectedRowValues, true); + assertResultsIgnoreOrder(rowIter, expectedRowValues2, true); } @Test From 949758d18c555b4097d9eeb7156901b2c9b7b379 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Thu, 7 Aug 2025 14:53:22 +0800 Subject: [PATCH 20/22] fix --- .../src/main/java/org/apache/fluss/types/RowType.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/fluss-common/src/main/java/org/apache/fluss/types/RowType.java b/fluss-common/src/main/java/org/apache/fluss/types/RowType.java index 99262c1fd2..3c1122f63c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/RowType.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/RowType.java @@ -265,14 +265,4 @@ public RowType build() { return new RowType(isNullable, fields); } } - - public DataField getField(String fieldName) { - for (DataField field : fields) { - if (field.getName().equals(fieldName)) { - return field; - } - } - - throw new RuntimeException("Cannot find field: " + fieldName); - } } From 7493f9537c37edae4a7cb6df1d9bb9617e616295 Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Thu, 7 Aug 2025 15:00:31 +0800 Subject: [PATCH 21/22] fix --- .../apache/fluss/flink/source/FlinkTableSourceITCase.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 5f44640181..3de0a680b7 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1189,7 +1189,7 @@ void testStreamingReadPartitionPushDownWithInExpr() throws Exception { .stream() .filter(s -> s.contains("2025") || s.contains("2026")) .collect(Collectors.toList()); - waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); + waitUntilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); String plan = tEnv.explainSql("select * from partitioned_table_in where c in ('2025','2026')"); @@ -1247,7 +1247,7 @@ void testStreamingReadWithCombinedFiltersAndInExpr() throws Exception { } writeRows(conn, tablePath, rows, false); - waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); + waitUntilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "2027")); String plan = tEnv.explainSql( @@ -1291,7 +1291,7 @@ void testStreamingReadPartitionPushDownWithLikeExpr() throws Exception { allData.stream() .filter(s -> s.contains("2025") || s.contains("2026")) .collect(Collectors.toList()); - waitUtilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "3026")); + waitUntilAllBucketFinishSnapshot(admin, tablePath, Arrays.asList("2025", "2026", "3026")); String plan = tEnv.explainSql("select * from partitioned_table_like where c like '202%'"); assertThat(plan) @@ -1350,7 +1350,7 @@ void testStreamingReadPartitionComplexPushDown() throws Exception { allData.stream() .filter(s -> s.contains("v3") && !s.contains("2025, 2")) .collect(Collectors.toList()); - waitUtilAllBucketFinishSnapshot( + waitUntilAllBucketFinishSnapshot( admin, tablePath, Arrays.asList("2025$1", "2025$2", "2026$1")); String plan = From 2a3892b955272657d548ede46ef61dd0df19585f Mon Sep 17 00:00:00 2001 From: Alibaba-HZY <19858181030@163.com> Date: Wed, 10 Sep 2025 17:45:39 +0800 Subject: [PATCH 22/22] [flink]fix and add log --- .../java/com/alibaba/fluss/predicate/And.java | 67 --- .../alibaba/fluss/predicate/CompareUtils.java | 60 --- .../fluss/predicate/CompoundPredicate.java | 114 ----- .../com/alibaba/fluss/predicate/Contains.java | 62 --- .../com/alibaba/fluss/predicate/EndsWith.java | 65 --- .../com/alibaba/fluss/predicate/Equal.java | 58 --- .../com/alibaba/fluss/predicate/FieldRef.java | 79 --- .../fluss/predicate/FunctionVisitor.java | 83 ---- .../fluss/predicate/GreaterOrEqual.java | 58 --- .../alibaba/fluss/predicate/GreaterThan.java | 58 --- .../java/com/alibaba/fluss/predicate/In.java | 83 ---- .../alibaba/fluss/predicate/IsNotNull.java | 55 --- .../com/alibaba/fluss/predicate/IsNull.java | 55 --- .../alibaba/fluss/predicate/LeafFunction.java | 65 --- .../fluss/predicate/LeafPredicate.java | 179 ------- .../fluss/predicate/LeafUnaryFunction.java | 53 -- .../alibaba/fluss/predicate/LessOrEqual.java | 58 --- .../com/alibaba/fluss/predicate/LessThan.java | 58 --- .../com/alibaba/fluss/predicate/NotEqual.java | 58 --- .../com/alibaba/fluss/predicate/NotIn.java | 83 ---- .../NullFalseLeafBinaryFunction.java | 61 --- .../java/com/alibaba/fluss/predicate/Or.java | 67 --- .../predicate/PartitionPredicateVisitor.java | 51 -- .../alibaba/fluss/predicate/Predicate.java | 50 -- .../fluss/predicate/PredicateBuilder.java | 454 ------------------ .../fluss/predicate/PredicateVisitor.java | 30 -- .../alibaba/fluss/predicate/StartsWith.java | 69 --- .../predicate/UnsupportedExpression.java | 29 -- .../apache/fluss/utils/BinaryStringUtils.java | 14 +- .../org/apache/fluss/utils/DateTimeUtils.java | 4 +- .../fluss/predicate/PredicateBuilderTest.java | 85 ---- .../fluss/predicate/PredicateTest.java | 424 ---------------- .../fluss/flink/row/FlinkAsFlussRow.java | 12 +- .../fluss/flink/source/FlinkSource.java | 71 ++- .../fluss/flink/source/FlinkTableSource.java | 85 +++- .../flink/source/PredicateConverter.java | 297 ++++++++++++ .../enumerator/FlinkSourceEnumerator.java | 161 ++++--- .../flink/source/FlinkTableSourceITCase.java | 2 +- .../enumerator/FlinkSourceEnumeratorTest.java | 1 + fluss-test-coverage/pom.xml | 53 +- 40 files changed, 586 insertions(+), 2885 deletions(-) delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java delete mode 100644 fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java delete mode 100644 fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java delete mode 100644 fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/PredicateConverter.java diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java deleted file mode 100644 index 6793ad8306..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/And.java +++ /dev/null @@ -1,67 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.InternalRow; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link CompoundPredicate.Function} to eval and. */ -public class And extends CompoundPredicate.Function { - - private static final long serialVersionUID = 1L; - - public static final And INSTANCE = new And(); - - private And() {} - - @Override - public boolean test(InternalRow row, List children) { - for (Predicate child : children) { - if (!child.test(row)) { - return false; - } - } - return true; - } - - @Override - public Optional negate(List children) { - List negatedChildren = new ArrayList<>(); - for (Predicate child : children) { - Optional negatedChild = child.negate(); - if (negatedChild.isPresent()) { - negatedChildren.add(negatedChild.get()); - } else { - return Optional.empty(); - } - } - return Optional.of(new CompoundPredicate(Or.INSTANCE, negatedChildren)); - } - - @Override - public T visit(FunctionVisitor visitor, List children) { - return visitor.visitAnd(children); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java deleted file mode 100644 index 50380abf89..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompareUtils.java +++ /dev/null @@ -1,60 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.BinaryString; -import com.alibaba.fluss.types.DataType; - -import static java.lang.Math.min; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Utils for comparator. */ -public class CompareUtils { - private CompareUtils() {} - - public static int compareLiteral(DataType type, Object v1, Object v2) { - if (v1 instanceof Comparable) { - // because BinaryString can not serialize so v1 or v2 may be BinaryString convert to - // String for compare - if (v1 instanceof BinaryString) { - v1 = ((BinaryString) v1).toString(); - } - if (v2 instanceof BinaryString) { - v2 = ((BinaryString) v2).toString(); - } - return ((Comparable) v1).compareTo(v2); - } else if (v1 instanceof byte[]) { - return compare((byte[]) v1, (byte[]) v2); - } else { - throw new RuntimeException("Unsupported type: " + type); - } - } - - private static int compare(byte[] first, byte[] second) { - for (int x = 0; x < min(first.length, second.length); x++) { - int cmp = first[x] - second[x]; - if (cmp != 0) { - return cmp; - } - } - return first.length - second.length; - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java deleted file mode 100644 index 449598f2f9..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/CompoundPredicate.java +++ /dev/null @@ -1,114 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.InternalRow; - -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** - * Non-leaf node in a {@link Predicate} tree. Its evaluation result depends on the results of its - * children. - */ -public class CompoundPredicate implements Predicate { - - private final Function function; - private final List children; - - public CompoundPredicate(Function function, List children) { - this.function = function; - this.children = children; - } - - public Function function() { - return function; - } - - public List children() { - return children; - } - - @Override - public boolean test(InternalRow row) { - return function.test(row, children); - } - - @Override - public Optional negate() { - return function.negate(children); - } - - @Override - public T visit(PredicateVisitor visitor) { - return visitor.visit(this); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof CompoundPredicate)) { - return false; - } - CompoundPredicate that = (CompoundPredicate) o; - return Objects.equals(function, that.function) && Objects.equals(children, that.children); - } - - @Override - public int hashCode() { - return Objects.hash(function, children); - } - - @Override - public String toString() { - return function + "(" + children + ")"; - } - - /** 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 Optional negate(List children); - - public abstract T visit(FunctionVisitor visitor, List children); - - @Override - public int hashCode() { - return this.getClass().getName().hashCode(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - return o != null && getClass() == o.getClass(); - } - - @Override - public String toString() { - return getClass().getSimpleName(); - } - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java deleted file mode 100644 index cfdaf0a06e..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Contains.java +++ /dev/null @@ -1,62 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like '%abc%'}. */ -public class Contains extends NullFalseLeafBinaryFunction { - - public static final Contains INSTANCE = new Contains(); - - private Contains() {} - - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - String fieldString = field.toString(); - return fieldString.contains((String) 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.visitContains(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java deleted file mode 100644 index a6af4c2e1d..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/EndsWith.java +++ /dev/null @@ -1,65 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** - * 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) { - String fieldString = field.toString(); - return fieldString.endsWith((String) 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.java deleted file mode 100644 index 26b1817492..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Equal.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval equal. */ -public class Equal extends NullFalseLeafBinaryFunction { - - public static final Equal INSTANCE = new Equal(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitEqual(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java deleted file mode 100644 index b9e179ddcd..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FieldRef.java +++ /dev/null @@ -1,79 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.io.Serializable; -import java.util.Objects; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A reference to a field in an input. */ -public class FieldRef implements Serializable { - - private static final long serialVersionUID = 1L; - - private final int index; - private final String name; - private final DataType type; - - public FieldRef(int index, String name, DataType type) { - this.index = index; - this.name = name; - this.type = type; - } - - public int index() { - return index; - } - - public String name() { - return name; - } - - public DataType type() { - return type; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - FieldRef fieldRef = (FieldRef) o; - return index == fieldRef.index - && Objects.equals(name, fieldRef.name) - && Objects.equals(type, fieldRef.type); - } - - @Override - public int hashCode() { - return Objects.hash(index, name, type); - } - - @Override - public String toString() { - return "FieldRef{" + "index=" + index + ", name='" + name + '\'' + ", type=" + type + '}'; - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java deleted file mode 100644 index 806f2de376..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/FunctionVisitor.java +++ /dev/null @@ -1,83 +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 com.alibaba.fluss.predicate; - -import java.util.List; -import java.util.stream.Collectors; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link PredicateVisitor} to visit functions. */ -public interface FunctionVisitor extends PredicateVisitor { - - @Override - default T visit(LeafPredicate predicate) { - return predicate.function().visit(this, predicate.fieldRef(), predicate.literals()); - } - - @Override - default T visit(CompoundPredicate predicate) { - return predicate - .function() - .visit( - this, - predicate.children().stream() - .map(p -> p.visit(this)) - .collect(Collectors.toList())); - } - - // ----------------- Unary functions ------------------------ - - T visitIsNotNull(FieldRef fieldRef); - - T visitIsNull(FieldRef fieldRef); - - // ----------------- Binary functions ------------------------ - - T visitStartsWith(FieldRef fieldRef, Object literal); - - T visitEndsWith(FieldRef fieldRef, Object literal); - - T visitContains(FieldRef fieldRef, Object literal); - - T visitLessThan(FieldRef fieldRef, Object literal); - - T visitGreaterOrEqual(FieldRef fieldRef, Object literal); - - T visitNotEqual(FieldRef fieldRef, Object literal); - - T visitLessOrEqual(FieldRef fieldRef, Object literal); - - T visitEqual(FieldRef fieldRef, Object literal); - - T visitGreaterThan(FieldRef fieldRef, Object literal); - - // ----------------- Other functions ------------------------ - - T visitIn(FieldRef fieldRef, List literals); - - T visitNotIn(FieldRef fieldRef, List literals); - - // ----------------- Compound functions ------------------------ - - T visitAnd(List children); - - T visitOr(List children); -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.java deleted file mode 100644 index 0a915355fb..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterOrEqual.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval greater or equal. */ -public class GreaterOrEqual extends NullFalseLeafBinaryFunction { - - public static final GreaterOrEqual INSTANCE = new GreaterOrEqual(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitGreaterOrEqual(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.java deleted file mode 100644 index f92c84c9bb..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/GreaterThan.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link LeafFunction} to eval greater. */ -public class GreaterThan extends NullFalseLeafBinaryFunction { - - public static final GreaterThan INSTANCE = new GreaterThan(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitGreaterThan(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java deleted file mode 100644 index 6697750ce2..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/In.java +++ /dev/null @@ -1,83 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java deleted file mode 100644 index 6249306b59..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNotNull.java +++ /dev/null @@ -1,55 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval is not null. */ -public class IsNotNull extends LeafUnaryFunction { - - public static final IsNotNull INSTANCE = new IsNotNull(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitIsNotNull(fieldRef); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java deleted file mode 100644 index d97c8cb433..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/IsNull.java +++ /dev/null @@ -1,55 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval is null. */ -public class IsNull extends LeafUnaryFunction { - - public static final IsNull INSTANCE = new IsNull(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitIsNull(fieldRef); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java deleted file mode 100644 index a897c902e6..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafFunction.java +++ /dev/null @@ -1,65 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.io.Serializable; -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** 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 - public int hashCode() { - return this.getClass().getName().hashCode(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - return o != null && getClass() == o.getClass(); - } - - public abstract T visit( - FunctionVisitor visitor, FieldRef fieldRef, List literals); - - @Override - public String toString() { - return getClass().getSimpleName(); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java deleted file mode 100644 index 35ef76e919..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafPredicate.java +++ /dev/null @@ -1,179 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.InternalRow; -import com.alibaba.fluss.types.DataType; -import com.alibaba.fluss.types.DecimalType; -import com.alibaba.fluss.types.LocalZonedTimestampType; -import com.alibaba.fluss.types.TimestampType; - -import java.util.List; -import java.util.Objects; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Leaf node of a {@link Predicate} tree. Compares a field in the row with literals. */ -public class LeafPredicate implements Predicate { - - private static final long serialVersionUID = 1L; - - private final LeafFunction function; - private final DataType type; - private final int fieldIndex; - private final String fieldName; - - private List literals; - - public LeafPredicate( - LeafFunction function, - DataType type, - int fieldIndex, - String fieldName, - List literals) { - this.function = function; - this.type = type; - this.fieldIndex = fieldIndex; - this.fieldName = fieldName; - this.literals = literals; - } - - public LeafFunction function() { - return function; - } - - public DataType type() { - return type; - } - - public int index() { - return fieldIndex; - } - - public String fieldName() { - return fieldName; - } - - public FieldRef fieldRef() { - return new FieldRef(fieldIndex, fieldName, type); - } - - public List literals() { - return literals; - } - - public LeafPredicate copyWithNewIndex(int fieldIndex) { - return new LeafPredicate(function, type, fieldIndex, fieldName, literals); - } - - @Override - public boolean test(InternalRow row) { - return function.test(type, get(row, fieldIndex, type), literals); - } - - @Override - public Optional negate() { - return function.negate() - .map(negate -> new LeafPredicate(negate, type, fieldIndex, fieldName, literals)); - } - - @Override - public T visit(PredicateVisitor visitor) { - return visitor.visit(this); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LeafPredicate that = (LeafPredicate) o; - return fieldIndex == that.fieldIndex - && Objects.equals(fieldName, that.fieldName) - && Objects.equals(function, that.function) - && Objects.equals(type, that.type) - && Objects.equals(literals, that.literals); - } - - @Override - public int hashCode() { - return Objects.hash(function, type, fieldIndex, fieldName, literals); - } - - @Override - public String toString() { - String literalsStr; - if (literals == null || literals.isEmpty()) { - literalsStr = ""; - } else if (literals.size() == 1) { - literalsStr = Objects.toString(literals.get(0)); - } else { - literalsStr = literals.toString(); - } - return literalsStr.isEmpty() - ? function + "(" + fieldName + ")" - : function + "(" + fieldName + ", " + literalsStr + ")"; - } - - public static Object get(InternalRow internalRow, int pos, DataType fieldType) { - if (internalRow.isNullAt(pos)) { - return null; - } - switch (fieldType.getTypeRoot()) { - case BOOLEAN: - return internalRow.getBoolean(pos); - case TINYINT: - return internalRow.getByte(pos); - case SMALLINT: - return internalRow.getShort(pos); - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - return internalRow.getInt(pos); - case BIGINT: - return internalRow.getLong(pos); - case TIMESTAMP_WITHOUT_TIME_ZONE: - TimestampType timestampType = (TimestampType) fieldType; - return internalRow.getTimestampNtz(pos, timestampType.getPrecision()); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - LocalZonedTimestampType lzTs = (LocalZonedTimestampType) fieldType; - return internalRow.getTimestampNtz(pos, lzTs.getPrecision()); - case FLOAT: - return internalRow.getFloat(pos); - case DOUBLE: - return internalRow.getDouble(pos); - case CHAR: - case STRING: - return internalRow.getString(pos); - case DECIMAL: - DecimalType decimalType = (DecimalType) fieldType; - return internalRow.getDecimal( - pos, decimalType.getPrecision(), decimalType.getScale()); - case BINARY: - return internalRow.getBytes(pos); - default: - throw new UnsupportedOperationException("Unsupported type: " + fieldType); - } - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java deleted file mode 100644 index e03710aa01..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LeafUnaryFunction.java +++ /dev/null @@ -1,53 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Function to test a field. */ -public abstract class LeafUnaryFunction extends LeafFunction { - - private static final long serialVersionUID = 1L; - - public abstract boolean test(DataType type, Object value); - - public abstract boolean test( - DataType type, long rowCount, Object min, Object max, Long nullCount); - - @Override - public boolean test(DataType type, Object value, List literals) { - return test(type, value); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - List literals) { - return test(type, rowCount, min, max, nullCount); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.java deleted file mode 100644 index 45dd3fda65..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessOrEqual.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ -public class LessOrEqual extends NullFalseLeafBinaryFunction { - - public static final LessOrEqual INSTANCE = new LessOrEqual(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitLessOrEqual(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.java deleted file mode 100644 index ed1ee70c3a..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/LessThan.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval less or equal. */ -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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitLessThan(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.java deleted file mode 100644 index bca203a22b..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotEqual.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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link NullFalseLeafBinaryFunction} to eval not equal. */ -public class NotEqual extends NullFalseLeafBinaryFunction { - - public static final NotEqual INSTANCE = new NotEqual(); - - 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); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitNotEqual(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java deleted file mode 100644 index fa7cbfef3e..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NotIn.java +++ /dev/null @@ -1,83 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -import static com.alibaba.fluss.predicate.CompareUtils.compareLiteral; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java deleted file mode 100644 index da170834eb..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/NullFalseLeafBinaryFunction.java +++ /dev/null @@ -1,61 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** 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/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java deleted file mode 100644 index b348a54781..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Or.java +++ /dev/null @@ -1,67 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.InternalRow; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A {@link CompoundPredicate.Function} to eval or. */ -public class Or extends CompoundPredicate.Function { - - private static final long serialVersionUID = 1L; - - public static final Or INSTANCE = new Or(); - - private Or() {} - - @Override - public boolean test(InternalRow row, List children) { - for (Predicate child : children) { - if (child.test(row)) { - return true; - } - } - return false; - } - - @Override - public Optional negate(List children) { - List negatedChildren = new ArrayList<>(); - for (Predicate child : children) { - Optional negatedChild = child.negate(); - if (negatedChild.isPresent()) { - negatedChildren.add(negatedChild.get()); - } else { - return Optional.empty(); - } - } - return Optional.of(new CompoundPredicate(And.INSTANCE, negatedChildren)); - } - - @Override - public T visit(FunctionVisitor visitor, List children) { - return visitor.visitOr(children); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java deleted file mode 100644 index 6318aba9c8..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PartitionPredicateVisitor.java +++ /dev/null @@ -1,51 +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 com.alibaba.fluss.predicate; - -import java.util.List; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** Visit the predicate and check if it only contains partition key's predicate. */ -public class PartitionPredicateVisitor implements PredicateVisitor { - - private final List partitionKeys; - - public PartitionPredicateVisitor(List partitionKeys) { - this.partitionKeys = partitionKeys; - } - - @Override - public Boolean visit(LeafPredicate predicate) { - return partitionKeys.contains(predicate.fieldName()); - } - - @Override - public Boolean visit(CompoundPredicate predicate) { - for (Predicate child : predicate.children()) { - Boolean matched = child.visit(this); - - if (!matched) { - return false; - } - } - return true; - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java deleted file mode 100644 index 7083a33d5b..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/Predicate.java +++ /dev/null @@ -1,50 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.InternalRow; - -import java.io.Serializable; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** - * Predicate which returns Boolean and provides testing by stats. - * - * @see PredicateBuilder - * @since 0.4.0 - */ -public interface Predicate extends Serializable { - - /** - * Now only support test based on the specific input row. Todo: boolean test(long rowCount, - * InternalRow minValues, InternalRow maxValues, InternalArray nullCounts); Test based on the - * specific input row. - * - * @return return true when hit, false when not hit. - */ - boolean test(InternalRow row); - - /** @return the negation predicate of this predicate if possible. */ - Optional negate(); - - T visit(PredicateVisitor visitor); -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java deleted file mode 100644 index 975954c2a7..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateBuilder.java +++ /dev/null @@ -1,454 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.BinaryString; -import com.alibaba.fluss.row.Decimal; -import com.alibaba.fluss.row.TimestampLtz; -import com.alibaba.fluss.row.TimestampNtz; -import com.alibaba.fluss.types.DataField; -import com.alibaba.fluss.types.DataType; -import com.alibaba.fluss.types.DecimalType; -import com.alibaba.fluss.types.RowType; -import com.alibaba.fluss.utils.TypeUtils; - -import javax.annotation.Nullable; - -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Timestamp; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; - -import static com.alibaba.fluss.utils.Preconditions.checkArgument; -import static java.util.Collections.singletonList; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** - * A utility class to create {@link Predicate} object for common filter conditions. - * - * @since 0.4.0 - */ -public class PredicateBuilder { - - private final RowType rowType; - private final List fieldNames; - - public PredicateBuilder(RowType rowType) { - this.rowType = rowType; - this.fieldNames = rowType.getFieldNames(); - } - - public int indexOf(String field) { - return fieldNames.indexOf(field); - } - - public Predicate equal(int idx, Object literal) { - return leaf(Equal.INSTANCE, idx, literal); - } - - public Predicate notEqual(int idx, Object literal) { - return leaf(NotEqual.INSTANCE, idx, literal); - } - - public Predicate lessThan(int idx, Object literal) { - return leaf(LessThan.INSTANCE, idx, literal); - } - - public Predicate lessOrEqual(int idx, Object literal) { - return leaf(LessOrEqual.INSTANCE, idx, literal); - } - - public Predicate greaterThan(int idx, Object literal) { - return leaf(GreaterThan.INSTANCE, idx, literal); - } - - public Predicate greaterOrEqual(int idx, Object literal) { - return leaf(GreaterOrEqual.INSTANCE, idx, literal); - } - - public Predicate isNull(int idx) { - return leaf(IsNull.INSTANCE, idx); - } - - public Predicate isNotNull(int idx) { - return leaf(IsNotNull.INSTANCE, idx); - } - - 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 contains(int idx, Object patternLiteral) { - return leaf(Contains.INSTANCE, idx, patternLiteral); - } - - public Predicate leaf(NullFalseLeafBinaryFunction function, int idx, Object literal) { - validateIndex(idx); - DataField field = rowType.getFields().get(idx); - return new LeafPredicate( - function, field.getType(), idx, field.getName(), singletonList(literal)); - } - - public Predicate leaf(LeafUnaryFunction function, int idx) { - validateIndex(idx); - DataField field = rowType.getFields().get(idx); - return new LeafPredicate( - function, field.getType(), idx, field.getName(), Collections.emptyList()); - } - - public Predicate in(int idx, List literals) { - validateIndex(idx); - // In the IN predicate, 20 literals are critical for performance. - // If there are more than 20 literals, the performance will decrease. - if (literals.size() > 20) { - DataField field = rowType.getFields().get(idx); - return new LeafPredicate(In.INSTANCE, field.getType(), idx, field.getName(), literals); - } - - List equals = new ArrayList<>(literals.size()); - for (Object literal : literals) { - equals.add(equal(idx, literal)); - } - return or(equals); - } - - public Predicate notIn(int idx, List literals) { - return in(idx, literals).negate().get(); - } - - public Predicate between(int idx, Object includedLowerBound, Object includedUpperBound) { - return new CompoundPredicate( - And.INSTANCE, - Arrays.asList( - greaterOrEqual(idx, includedLowerBound), - lessOrEqual(idx, includedUpperBound))); - } - - public static Predicate and(Predicate... predicates) { - return and(Arrays.asList(predicates)); - } - - public static Predicate and(List predicates) { - checkArgument( - predicates.size() > 0, - "There must be at least 1 inner predicate to construct an AND predicate"); - if (predicates.size() == 1) { - return predicates.get(0); - } - return predicates.stream() - .reduce((a, b) -> new CompoundPredicate(And.INSTANCE, Arrays.asList(a, b))) - .get(); - } - - @Nullable - public static Predicate andNullable(Predicate... predicates) { - return andNullable(Arrays.asList(predicates)); - } - - @Nullable - public static Predicate andNullable(List predicates) { - predicates = predicates.stream().filter(Objects::nonNull).collect(Collectors.toList()); - if (predicates.isEmpty()) { - return null; - } - - return and(predicates); - } - - public static Predicate or(Predicate... predicates) { - return or(Arrays.asList(predicates)); - } - - public static Predicate or(List predicates) { - checkArgument( - predicates.size() > 0, - "There must be at least 1 inner predicate to construct an OR predicate"); - return predicates.stream() - .reduce((a, b) -> new CompoundPredicate(Or.INSTANCE, Arrays.asList(a, b))) - .get(); - } - - public static List splitAnd(@Nullable Predicate predicate) { - if (predicate == null) { - return Collections.emptyList(); - } - List result = new ArrayList<>(); - splitCompound(And.INSTANCE, predicate, result); - return result; - } - - public static List splitOr(@Nullable Predicate predicate) { - if (predicate == null) { - return Collections.emptyList(); - } - List result = new ArrayList<>(); - splitCompound(Or.INSTANCE, predicate, result); - return result; - } - - private static void splitCompound( - CompoundPredicate.Function function, Predicate predicate, List result) { - if (predicate instanceof CompoundPredicate - && ((CompoundPredicate) predicate).function().equals(function)) { - for (Predicate child : ((CompoundPredicate) predicate).children()) { - splitCompound(function, child, result); - } - } else { - result.add(predicate); - } - } - - private void validateIndex(int idx) { - if (idx < 0 || idx >= rowType.getFieldCount()) { - throw new UnsupportedExpression("idx is not valid"); - } - } - - public static Object convertJavaObject(DataType literalType, Object o) { - if (o == null) { - return null; - } - switch (literalType.getTypeRoot()) { - case BOOLEAN: - return o; - case BIGINT: - return ((Number) o).longValue(); - case DOUBLE: - return ((Number) o).doubleValue(); - case TINYINT: - return ((Number) o).byteValue(); - case SMALLINT: - return ((Number) o).shortValue(); - case INTEGER: - return ((Number) o).intValue(); - case FLOAT: - return ((Number) o).floatValue(); - case STRING: - return BinaryString.fromString(o.toString()); - case DATE: - // Hive uses `java.sql.Date.valueOf(lit.toString());` to convert a literal to Date - // Which uses `java.util.Date()` internally to create the object and that uses the - // TimeZone.getDefaultRef() - // To get back the expected date we have to use the LocalDate which gets rid of the - // TimeZone misery as it uses the year/month/day to generate the object - LocalDate localDate; - if (o instanceof java.sql.Timestamp) { - localDate = ((java.sql.Timestamp) o).toLocalDateTime().toLocalDate(); - } else if (o instanceof Date) { - localDate = ((Date) o).toLocalDate(); - } else if (o instanceof LocalDate) { - localDate = (LocalDate) o; - } else { - throw new UnsupportedOperationException( - "Unexpected date literal of class " + o.getClass().getName()); - } - LocalDate epochDay = - Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC).toLocalDate(); - return (int) ChronoUnit.DAYS.between(epochDay, localDate); - case TIME_WITHOUT_TIME_ZONE: - LocalTime localTime; - if (o instanceof java.sql.Time) { - localTime = ((java.sql.Time) o).toLocalTime(); - } else if (o instanceof LocalTime) { - localTime = (LocalTime) o; - } else { - throw new UnsupportedOperationException( - "Unexpected time literal of class " + o.getClass().getName()); - } - // return millis of a day - return (int) (localTime.toNanoOfDay() / 1_000_000); - case DECIMAL: - DecimalType decimalType = (DecimalType) literalType; - int precision = decimalType.getPrecision(); - int scale = decimalType.getScale(); - return Decimal.fromBigDecimal((BigDecimal) o, precision, scale); - case TIMESTAMP_WITHOUT_TIME_ZONE: - if (o instanceof java.sql.Timestamp) { - LocalDateTime localDateTime = - ((Timestamp) o) - .toInstant() - .atZone(ZoneId.systemDefault()) - .toLocalDateTime(); - return TimestampNtz.fromLocalDateTime(localDateTime); - } else if (o instanceof Instant) { - Instant o1 = (Instant) o; - LocalDateTime dateTime = o1.atZone(ZoneId.systemDefault()).toLocalDateTime(); - return TimestampNtz.fromLocalDateTime(dateTime); - } else if (o instanceof LocalDateTime) { - return TimestampNtz.fromLocalDateTime((LocalDateTime) o); - } else { - throw new UnsupportedOperationException( - String.format( - "Unsupported class %s for timestamp without timezone ", - o.getClass())); - } - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - if (o instanceof java.sql.Timestamp) { - java.sql.Timestamp timestamp = (java.sql.Timestamp) o; - return TimestampLtz.fromInstant(timestamp.toInstant()); - } else if (o instanceof Instant) { - return TimestampLtz.fromInstant((Instant) o); - } else { - throw new UnsupportedOperationException( - String.format( - "Unsupported class %s for timestamp with local time zone ", - o.getClass())); - } - default: - throw new UnsupportedOperationException( - "Unsupported predicate leaf type " + literalType.getTypeRoot().name()); - } - } - - public static List pickTransformFieldMapping( - List predicates, List inputFields, List pickedFields) { - return pickTransformFieldMapping( - predicates, inputFields.stream().mapToInt(pickedFields::indexOf).toArray()); - } - - public static List pickTransformFieldMapping( - List predicates, int[] fieldIdxMapping) { - List pick = new ArrayList<>(); - for (Predicate p : predicates) { - Optional mapped = transformFieldMapping(p, fieldIdxMapping); - mapped.ifPresent(pick::add); - } - return pick; - } - - public static Optional transformFieldMapping( - Predicate predicate, int[] fieldIdxMapping) { - if (predicate instanceof CompoundPredicate) { - CompoundPredicate compoundPredicate = (CompoundPredicate) predicate; - List children = new ArrayList<>(); - for (Predicate child : compoundPredicate.children()) { - Optional mapped = transformFieldMapping(child, fieldIdxMapping); - if (mapped.isPresent()) { - children.add(mapped.get()); - } else { - return Optional.empty(); - } - } - return Optional.of(new CompoundPredicate(compoundPredicate.function(), children)); - } else { - LeafPredicate leafPredicate = (LeafPredicate) predicate; - int mapped = fieldIdxMapping[leafPredicate.index()]; - if (mapped >= 0) { - return Optional.of( - new LeafPredicate( - leafPredicate.function(), - leafPredicate.type(), - mapped, - leafPredicate.fieldName(), - leafPredicate.literals())); - } else { - return Optional.empty(); - } - } - } - - public static boolean containsFields(Predicate predicate, Set fields) { - if (predicate instanceof CompoundPredicate) { - for (Predicate child : ((CompoundPredicate) predicate).children()) { - if (containsFields(child, fields)) { - return true; - } - } - return false; - } else { - LeafPredicate leafPredicate = (LeafPredicate) predicate; - return fields.contains(leafPredicate.fieldName()); - } - } - - public static List excludePredicateWithFields( - @Nullable List predicates, Set fields) { - if (predicates == null || predicates.isEmpty() || fields.isEmpty()) { - return predicates; - } - return predicates.stream() - .filter(f -> !containsFields(f, fields)) - .collect(Collectors.toList()); - } - - @Nullable - public static Predicate partition( - Map map, RowType rowType, String defaultPartValue) { - Map internalValues = convertSpecToInternal(map, rowType, defaultPartValue); - List fieldNames = rowType.getFieldNames(); - Predicate predicate = null; - PredicateBuilder builder = new PredicateBuilder(rowType); - for (Map.Entry entry : internalValues.entrySet()) { - int idx = fieldNames.indexOf(entry.getKey()); - Object literal = internalValues.get(entry.getKey()); - Predicate predicateTemp = - literal == null ? builder.isNull(idx) : builder.equal(idx, literal); - if (predicate == null) { - predicate = predicateTemp; - } else { - predicate = PredicateBuilder.and(predicate, predicateTemp); - } - } - return predicate; - } - - public static Predicate partitions( - List> partitions, RowType rowType, String defaultPartValue) { - return PredicateBuilder.or( - partitions.stream() - .map(p -> PredicateBuilder.partition(p, rowType, defaultPartValue)) - .toArray(Predicate[]::new)); - } - - public static Map convertSpecToInternal( - Map spec, RowType partType, String defaultPartValue) { - Map partValues = new LinkedHashMap<>(); - for (Map.Entry entry : spec.entrySet()) { - partValues.put( - entry.getKey(), - defaultPartValue.equals(entry.getValue()) - ? null - : TypeUtils.castFromString( - entry.getValue(), partType.getField(entry.getKey()).getType())); - } - return partValues; - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java deleted file mode 100644 index 3e20e9c92a..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/PredicateVisitor.java +++ /dev/null @@ -1,30 +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 com.alibaba.fluss.predicate; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** A visitor to visit {@link Predicate}. */ -public interface PredicateVisitor { - - T visit(LeafPredicate predicate); - - T visit(CompoundPredicate predicate); -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java deleted file mode 100644 index e3baa41fa9..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/StartsWith.java +++ /dev/null @@ -1,69 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.types.DataType; - -import java.util.List; -import java.util.Optional; - -/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. */ - -/** - * A {@link NullFalseLeafBinaryFunction} to evaluate {@code filter like 'abc%' or filter like - * 'abc_'}. - */ -public class StartsWith extends NullFalseLeafBinaryFunction { - - public static final StartsWith INSTANCE = new StartsWith(); - - private StartsWith() {} - - @Override - public boolean test(DataType type, Object field, Object patternLiteral) { - String fieldString = field.toString(); - return fieldString.startsWith((String) patternLiteral); - } - - @Override - public boolean test( - DataType type, - long rowCount, - Object min, - Object max, - Long nullCount, - Object patternLiteral) { - String minStr = min.toString(); - String maxStr = max.toString(); - String pattern = patternLiteral.toString(); - return (minStr.startsWith(pattern) || minStr.compareTo(pattern) <= 0) - && (maxStr.startsWith(pattern) || maxStr.compareTo(pattern) >= 0); - } - - @Override - public Optional negate() { - return Optional.empty(); - } - - @Override - public T visit(FunctionVisitor visitor, FieldRef fieldRef, List literals) { - return visitor.visitStartsWith(fieldRef, literals.get(0)); - } -} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java b/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java deleted file mode 100644 index 74b50f2f4e..0000000000 --- a/fluss-common/src/main/java/com/alibaba/fluss/predicate/UnsupportedExpression.java +++ /dev/null @@ -1,29 +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 com.alibaba.fluss.predicate; - -/** Encounter an unsupported expression, the caller can choose to ignore this filter branch. */ -public class UnsupportedExpression extends RuntimeException { - public UnsupportedExpression(String message) { - super(message); - } - - public UnsupportedExpression() { - super(); - } -} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java index 8c61a10f6f..6275182c4c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java @@ -17,9 +17,9 @@ package org.apache.fluss.utils; -import com.alibaba.fluss.row.BinaryString; -import com.alibaba.fluss.row.TimestampLtz; -import com.alibaba.fluss.row.TimestampNtz; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; import java.time.DateTimeException; import java.util.List; @@ -76,9 +76,9 @@ public static TimestampNtz toTimestampNtz(BinaryString input, int precision) return DateTimeUtils.parseTimestampData(input.toString(), precision); } - /** Used by {@code CAST(x as TIMESTAMPLTZ)}. */ - public static TimestampLtz toTimestampltz(BinaryString input, int precision, TimeZone timeZone) - throws DateTimeException { - return DateTimeUtils.parseTimestampData(input.toString(), precision, timeZone); + /** Used by {@code CAST(x as TIMESTAMP_LTZ)}. */ + public static TimestampLtz toTimestampLtz( + BinaryString input, int precision, TimeZone localTimeZone) throws DateTimeException { + return DateTimeUtils.parseTimestampData(input.toString(), precision, localTimeZone); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java index fd645bb944..9106993eae 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/DateTimeUtils.java @@ -17,8 +17,8 @@ package org.apache.fluss.utils; -import com.alibaba.fluss.row.TimestampLtz; -import com.alibaba.fluss.row.TimestampNtz; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; import java.time.DateTimeException; import java.time.LocalDate; diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java deleted file mode 100644 index acb193fe3e..0000000000 --- a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateBuilderTest.java +++ /dev/null @@ -1,85 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.GenericRow; -import com.alibaba.fluss.types.IntType; -import com.alibaba.fluss.types.RowType; - -import org.junit.jupiter.api.Test; - -import java.util.Arrays; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link PredicateBuilder}. */ -public class PredicateBuilderTest { - - @Test - 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); - } - - @Test - 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); - } - - @Test - public void testSplitAnd() { - PredicateBuilder builder = - new PredicateBuilder( - RowType.of( - new IntType(), - new IntType(), - new IntType(), - new IntType(), - new IntType(), - new IntType(), - new IntType())); - - Predicate child1 = - PredicateBuilder.or(builder.isNull(0), builder.isNull(1), builder.isNull(2)); - Predicate child2 = - PredicateBuilder.and(builder.isNull(3), builder.isNull(4), builder.isNull(5)); - Predicate child3 = builder.isNull(6); - - assertThat(PredicateBuilder.splitAnd(PredicateBuilder.and(child1, child2, child3))) - .isEqualTo( - Arrays.asList( - child1, - builder.isNull(3), - builder.isNull(4), - builder.isNull(5), - child3)); - } -} diff --git a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java b/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java deleted file mode 100644 index 427fec8a82..0000000000 --- a/fluss-common/src/test/java/com/alibaba/fluss/predicate/PredicateTest.java +++ /dev/null @@ -1,424 +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 com.alibaba.fluss.predicate; - -import com.alibaba.fluss.row.GenericRow; -import com.alibaba.fluss.types.IntType; -import com.alibaba.fluss.types.RowType; -import com.alibaba.fluss.types.StringType; - -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static com.alibaba.fluss.row.BinaryString.fromString; -import static org.assertj.core.api.Assertions.assertThat; - -/** Test for {@link Predicate}s. */ -public class PredicateTest { - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.notEqual(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.equal(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.lessOrEqual(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.lessThan(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.greaterOrEqual(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.greaterThan(0, 5)); - } - - @Test - 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); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.isNotNull(0)); - } - - @Test - 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(predicate.negate().orElse(null)).isEqualTo(builder.isNull(0)); - } - - @Test - public void testIn() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - 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); - } - - @Test - public void testInNull() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - 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); - } - - @Test - public void testNotIn() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - 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); - } - - @Test - public void testNotInNull() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - 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); - } - - @Test - public void testEndsWith() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); - Predicate predicate = builder.endsWith(0, ("bcc")); - GenericRow row = GenericRow.of(fromString("aabbcc")); - - assertThat(predicate.test(row)).isEqualTo(true); - } - - @Test - public void testStartWith() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); - Predicate predicate = builder.startsWith(0, ("aab")); - GenericRow row = GenericRow.of(fromString("aabbcc")); - - assertThat(predicate.test(row)).isEqualTo(true); - } - - @Test - public void testContainsWith() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new StringType())); - Predicate predicate = builder.contains(0, ("def")); - GenericRow row1 = GenericRow.of(fromString("aabbdefcc")); - GenericRow row2 = GenericRow.of(fromString("aabbdcefcc")); - - assertThat(predicate.test(row1)).isEqualTo(true); - assertThat(predicate.test(row2)).isEqualTo(false); - } - - @Test - public void testLargeIn() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - List literals = new ArrayList<>(); - literals.add(1); - literals.add(3); - for (int i = 10; i < 30; i++) { - literals.add(i); - } - 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); - } - - @Test - public void testLargeInNull() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - List literals = new ArrayList<>(); - literals.add(1); - literals.add(null); - literals.add(3); - for (int i = 10; i < 30; i++) { - literals.add(i); - } - 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); - } - - @Test - public void testLargeNotIn() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - List literals = new ArrayList<>(); - literals.add(1); - literals.add(3); - for (int i = 10; i < 30; i++) { - literals.add(i); - } - 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); - } - - @Test - public void testLargeNotInNull() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - List literals = new ArrayList<>(); - literals.add(1); - literals.add(null); - literals.add(3); - for (int i = 10; i < 30; i++) { - literals.add(i); - } - 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); - } - - @Test - 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(predicate.negate().orElse(null)) - .isEqualTo(PredicateBuilder.or(builder.notEqual(0, 3), builder.notEqual(1, 5))); - } - - @Test - 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(predicate.negate().orElse(null)) - .isEqualTo(PredicateBuilder.and(builder.notEqual(0, 3), builder.notEqual(1, 5))); - } - - @Test - public void testUnknownStats() { - PredicateBuilder builder = new PredicateBuilder(RowType.of(new IntType())); - Predicate predicate = builder.equal(0, 5); - } - - @Test - public void testPredicateToString() { - PredicateBuilder builder1 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p1 = builder1.equal(0, 5); - assertThat(p1.toString()).isEqualTo("Equal(f0, 5)"); - - PredicateBuilder builder2 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p2 = builder2.greaterThan(0, 5); - assertThat(p2.toString()).isEqualTo("GreaterThan(f0, 5)"); - - PredicateBuilder builder3 = new PredicateBuilder(RowType.of(new IntType(), new IntType())); - Predicate p3 = PredicateBuilder.and(builder3.equal(0, 3), builder3.equal(1, 5)); - assertThat(p3.toString()).isEqualTo("And([Equal(f0, 3), Equal(f1, 5)])"); - - PredicateBuilder builder4 = new PredicateBuilder(RowType.of(new IntType(), new IntType())); - Predicate p4 = PredicateBuilder.or(builder4.equal(0, 3), builder4.equal(1, 5)); - assertThat(p4.toString()).isEqualTo("Or([Equal(f0, 3), Equal(f1, 5)])"); - - PredicateBuilder builder5 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p5 = builder5.isNotNull(0); - assertThat(p5.toString()).isEqualTo("IsNotNull(f0)"); - - PredicateBuilder builder6 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p6 = builder6.in(0, Arrays.asList(1, null, 3, 4)); - assertThat(p6.toString()) - .isEqualTo( - "Or([Or([Or([Equal(f0, 1), Equal(f0, null)]), Equal(f0, 3)]), Equal(f0, 4)])"); - - PredicateBuilder builder7 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p7 = builder7.notIn(0, Arrays.asList(1, null, 3, 4)); - assertThat(p7.toString()) - .isEqualTo( - "And([And([And([NotEqual(f0, 1), NotEqual(f0, null)]), NotEqual(f0, 3)]), NotEqual(f0, 4)])"); - - PredicateBuilder builder8 = new PredicateBuilder(RowType.of(new IntType())); - List literals = new ArrayList<>(); - for (int i = 1; i <= 21; i++) { - literals.add(i); - } - Predicate p8 = builder8.in(0, literals); - assertThat(p8.toString()) - .isEqualTo( - "In(f0, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21])"); - - PredicateBuilder builder9 = new PredicateBuilder(RowType.of(new IntType())); - Predicate p9 = builder9.notIn(0, literals); - assertThat(p9.toString()) - .isEqualTo( - "NotIn(f0, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21])"); - } -} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java index fbe74c49d7..3593d1f106 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java @@ -17,12 +17,12 @@ package org.apache.fluss.flink.row; -import com.alibaba.fluss.flink.utils.FlinkConversions; -import com.alibaba.fluss.row.BinaryString; -import com.alibaba.fluss.row.Decimal; -import com.alibaba.fluss.row.InternalRow; -import com.alibaba.fluss.row.TimestampLtz; -import com.alibaba.fluss.row.TimestampNtz; +import org.apache.fluss.flink.utils.FlinkConversions; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericRowData; diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java index e72325cfe0..aee2dde054 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkSource.java @@ -17,22 +17,24 @@ package org.apache.fluss.flink.source; -import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.flink.source.deserializer.DeserializerInitContextImpl; -import com.alibaba.fluss.flink.source.deserializer.FlussDeserializationSchema; -import com.alibaba.fluss.flink.source.emitter.FlinkRecordEmitter; -import com.alibaba.fluss.flink.source.enumerator.FlinkSourceEnumerator; -import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer; -import com.alibaba.fluss.flink.source.metrics.FlinkSourceReaderMetrics; -import com.alibaba.fluss.flink.source.reader.FlinkSourceReader; -import com.alibaba.fluss.flink.source.reader.RecordAndPos; -import com.alibaba.fluss.flink.source.split.SourceSplitBase; -import com.alibaba.fluss.flink.source.split.SourceSplitSerializer; -import com.alibaba.fluss.flink.source.state.FlussSourceEnumeratorStateSerializer; -import com.alibaba.fluss.flink.source.state.SourceEnumeratorState; -import com.alibaba.fluss.metadata.TablePath; -import com.alibaba.fluss.predicate.Predicate; -import com.alibaba.fluss.types.RowType; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.source.deserializer.DeserializerInitContextImpl; +import org.apache.fluss.flink.source.deserializer.FlussDeserializationSchema; +import org.apache.fluss.flink.source.emitter.FlinkRecordEmitter; +import org.apache.fluss.flink.source.enumerator.FlinkSourceEnumerator; +import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +import org.apache.fluss.flink.source.metrics.FlinkSourceReaderMetrics; +import org.apache.fluss.flink.source.reader.FlinkSourceReader; +import org.apache.fluss.flink.source.reader.RecordAndPos; +import org.apache.fluss.flink.source.split.SourceSplitBase; +import org.apache.fluss.flink.source.split.SourceSplitSerializer; +import org.apache.fluss.flink.source.state.FlussSourceEnumeratorStateSerializer; +import org.apache.fluss.flink.source.state.SourceEnumeratorState; +import org.apache.fluss.lake.source.LakeSource; +import org.apache.fluss.lake.source.LakeSplit; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.types.RowType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -66,6 +68,8 @@ public class FlinkSource private Predicate partitionFilters; + private final @Nullable LakeSource lakeSource; + public FlinkSource( Configuration flussConf, TablePath tablePath, @@ -78,6 +82,34 @@ public FlinkSource( FlussDeserializationSchema deserializationSchema, boolean streaming, Predicate partitionFilters) { + this( + flussConf, + tablePath, + hasPrimaryKey, + isPartitioned, + sourceOutputType, + projectedFields, + offsetsInitializer, + scanPartitionDiscoveryIntervalMs, + deserializationSchema, + streaming, + partitionFilters, + null); + } + + public FlinkSource( + Configuration flussConf, + TablePath tablePath, + boolean hasPrimaryKey, + boolean isPartitioned, + RowType sourceOutputType, + @Nullable int[] projectedFields, + OffsetsInitializer offsetsInitializer, + long scanPartitionDiscoveryIntervalMs, + FlussDeserializationSchema deserializationSchema, + boolean streaming, + Predicate partitionFilters, + LakeSource lakeSource) { this.flussConf = flussConf; this.tablePath = tablePath; this.hasPrimaryKey = hasPrimaryKey; @@ -89,6 +121,7 @@ public FlinkSource( this.deserializationSchema = deserializationSchema; this.streaming = streaming; this.partitionFilters = partitionFilters; + this.lakeSource = lakeSource; } @Override @@ -108,7 +141,8 @@ public SplitEnumerator createEnumerator( offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters); + partitionFilters, + lakeSource); } @Override @@ -127,7 +161,8 @@ public SplitEnumerator restoreEnumerator offsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - partitionFilters); + partitionFilters, + lakeSource); } @Override diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java index 627c8f9a97..23e4d7b414 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/FlinkTableSource.java @@ -17,23 +17,30 @@ package org.apache.fluss.flink.source; -import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.connector.flink.FlinkConnectorOptions; -import com.alibaba.fluss.connector.flink.source.enumerator.initializer.OffsetsInitializer; -import com.alibaba.fluss.connector.flink.source.lookup.FlinkAsyncLookupFunction; -import com.alibaba.fluss.connector.flink.source.lookup.FlinkLookupFunction; -import com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer; -import com.alibaba.fluss.connector.flink.utils.FlinkConnectorOptionsUtils; -import com.alibaba.fluss.connector.flink.utils.FlinkConversions; -import com.alibaba.fluss.connector.flink.utils.PushdownUtils; -import com.alibaba.fluss.connector.flink.utils.PushdownUtils.ValueConversion; -import com.alibaba.fluss.metadata.MergeEngineType; -import com.alibaba.fluss.metadata.TablePath; -import com.alibaba.fluss.predicate.PartitionPredicateVisitor; -import com.alibaba.fluss.predicate.Predicate; -import com.alibaba.fluss.predicate.PredicateBuilder; -import com.alibaba.fluss.predicate.PredicateVisitor; -import com.alibaba.fluss.types.RowType; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.FlinkConnectorOptions; +import org.apache.fluss.flink.source.deserializer.RowDataDeserializationSchema; +import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +import org.apache.fluss.flink.source.lookup.FlinkAsyncLookupFunction; +import org.apache.fluss.flink.source.lookup.FlinkLookupFunction; +import org.apache.fluss.flink.source.lookup.LookupNormalizer; +import org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils; +import org.apache.fluss.flink.utils.FlinkConversions; +import org.apache.fluss.flink.utils.PushdownUtils; +import org.apache.fluss.flink.utils.PushdownUtils.FieldEqual; +import org.apache.fluss.lake.source.LakeSource; +import org.apache.fluss.lake.source.LakeSplit; +import org.apache.fluss.metadata.MergeEngineType; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.predicate.GreaterOrEqual; +import org.apache.fluss.predicate.LeafPredicate; +import org.apache.fluss.predicate.PartitionPredicateVisitor; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.predicate.PredicateVisitor; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -68,6 +75,8 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.VarCharType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; @@ -82,7 +91,10 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.alibaba.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.flink.utils.LakeSourceUtils.createLakeSource; +import static org.apache.fluss.metadata.TableDescriptor.TIMESTAMP_COLUMN_NAME; +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.Preconditions.checkState; /** Flink table source to scan Fluss data. */ public class FlinkTableSource @@ -136,6 +148,10 @@ public class FlinkTableSource @Nullable protected Predicate partitionFilters; + private final Map tableOptions; + + @Nullable private LakeSource lakeSource; + public FlinkTableSource( TablePath tablePath, Configuration flussConfig, @@ -294,8 +310,10 @@ public boolean isBounded() { projectedFields, offsetsInitializer, scanPartitionDiscoveryIntervalMs, + new RowDataDeserializationSchema(), streaming, - partitionFilters); + partitionFilters, + enableLakeSource ? lakeSource : null); if (!streaming) { // return a bounded source provide to make planner happy, @@ -421,6 +439,7 @@ public DynamicTableSource copy() { source.singleRowFilter = singleRowFilter; source.modificationScanType = modificationScanType; source.partitionFilters = partitionFilters; + source.lakeSource = lakeSource; return source; } @@ -521,6 +540,34 @@ && hasPrimaryKey() } } partitionFilters = converted.isEmpty() ? null : PredicateBuilder.and(converted); + // lake source is not null + if (lakeSource != null) { + PredicateVisitor lakePredicateVisitor = + new PartitionPredicateVisitor(tableOutputType.getFieldNames()); + + List lakePredicates = new ArrayList<>(); + for (ResolvedExpression filter : filters) { + + Optional predicateOptional = + PredicateConverter.convert(tableOutputType, filter); + + if (predicateOptional.isPresent()) { + Predicate p = predicateOptional.get(); + lakePredicates.add(p); + } + } + + if (!lakePredicates.isEmpty()) { + final LakeSource.FilterPushDownResult filterPushDownResult = + lakeSource.withFilters(lakePredicates); + if (filterPushDownResult.acceptedPredicates().size() != lakePredicates.size()) { + LOG.info( + "LakeSource rejected some partition filters. Falling back to Flink-side filtering."); + // Flink will apply all filters to preserve correctness + return Result.of(Collections.emptyList(), filters); + } + } + } return Result.of(acceptedFilters, remainingFilters); } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/PredicateConverter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/PredicateConverter.java new file mode 100644 index 0000000000..73b36c0d15 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/PredicateConverter.java @@ -0,0 +1,297 @@ +/* + * 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.fluss.flink.source; + +import org.apache.fluss.flink.row.FlinkAsFlussRow; +import org.apache.fluss.flink.utils.FlinkConversions; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.predicate.PredicateBuilder; +import org.apache.fluss.predicate.UnsupportedExpression; +import org.apache.fluss.utils.TypeUtils; + +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.RowType; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.regex.Pattern; + +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsImplicitCast; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ +/** + * Convert {@link Expression} to {@link Predicate}. + * + *

For {@link FieldReferenceExpression}, please use name instead of index, if the project + * pushdown is before and the filter pushdown is after, the index of the filter will be projected. + */ +public class PredicateConverter implements ExpressionVisitor { + + private final PredicateBuilder builder; + + public PredicateConverter(RowType type) { + this(new PredicateBuilder(FlinkConversions.toFlussRowType(type))); + } + + public PredicateConverter(PredicateBuilder builder) { + this.builder = builder; + } + + /** Accepts simple LIKE patterns like "abc%". */ + private static final Pattern BEGIN_PATTERN = Pattern.compile("^[^%_]+%$"); + + private static final Pattern END_PATTERN = Pattern.compile("^%[^%_]+$"); + private static final Pattern CONTAINS_PATTERN = Pattern.compile("^%[^%_]+%$"); + + @Override + public Predicate visit(CallExpression call) { + FunctionDefinition func = call.getFunctionDefinition(); + List children = call.getChildren(); + + if (func == BuiltInFunctionDefinitions.AND) { + return PredicateBuilder.and(children.get(0).accept(this), children.get(1).accept(this)); + } else if (func == BuiltInFunctionDefinitions.OR) { + return PredicateBuilder.or(children.get(0).accept(this), children.get(1).accept(this)); + } else if (func == BuiltInFunctionDefinitions.EQUALS) { + return visitBiFunction(children, builder::equal, builder::equal); + } else if (func == BuiltInFunctionDefinitions.NOT_EQUALS) { + return visitBiFunction(children, builder::notEqual, builder::notEqual); + } else if (func == BuiltInFunctionDefinitions.GREATER_THAN) { + return visitBiFunction(children, builder::greaterThan, builder::lessThan); + } else if (func == BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL) { + return visitBiFunction(children, builder::greaterOrEqual, builder::lessOrEqual); + } else if (func == BuiltInFunctionDefinitions.LESS_THAN) { + return visitBiFunction(children, builder::lessThan, builder::greaterThan); + } else if (func == BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL) { + return visitBiFunction(children, builder::lessOrEqual, builder::greaterOrEqual); + } else if (func == BuiltInFunctionDefinitions.IN) { + FieldReferenceExpression fieldRefExpr = + extractFieldReference(children.get(0)).orElseThrow(UnsupportedExpression::new); + List literals = new ArrayList<>(); + for (int i = 1; i < children.size(); i++) { + literals.add(extractLiteral(fieldRefExpr.getOutputDataType(), children.get(i))); + } + return builder.in(builder.indexOf(fieldRefExpr.getName()), literals); + } else if (func == BuiltInFunctionDefinitions.IS_NULL) { + return extractFieldReference(children.get(0)) + .map(FieldReferenceExpression::getName) + .map(builder::indexOf) + .map(builder::isNull) + .orElseThrow(UnsupportedExpression::new); + } else if (func == BuiltInFunctionDefinitions.IS_NOT_NULL) { + return extractFieldReference(children.get(0)) + .map(FieldReferenceExpression::getName) + .map(builder::indexOf) + .map(builder::isNotNull) + .orElseThrow(UnsupportedExpression::new); + } else if (func == BuiltInFunctionDefinitions.LIKE) { + FieldReferenceExpression fieldRefExpr = + extractFieldReference(children.get(0)).orElseThrow(UnsupportedExpression::new); + if (fieldRefExpr + .getOutputDataType() + .getLogicalType() + .getTypeRoot() + .getFamilies() + .contains(LogicalTypeFamily.CHARACTER_STRING) + && builder.indexOf(fieldRefExpr.getName()) != -1) { + String sqlPattern = + Objects.requireNonNull( + extractLiteral( + fieldRefExpr.getOutputDataType(), children.get(1))) + .toString(); + String escape = + children.size() <= 2 + ? null + : Objects.requireNonNull( + extractLiteral( + fieldRefExpr.getOutputDataType(), + children.get(2))) + .toString(); + + if (escape == null) { + if (BEGIN_PATTERN.matcher(sqlPattern).matches()) { + String prefix = sqlPattern.substring(0, sqlPattern.length() - 1); + return builder.startsWith(builder.indexOf(fieldRefExpr.getName()), prefix); + } + if (END_PATTERN.matcher(sqlPattern).matches()) { + String suffix = sqlPattern.substring(1); + return builder.endsWith(builder.indexOf(fieldRefExpr.getName()), suffix); + } + if (CONTAINS_PATTERN.matcher(sqlPattern).matches() + && sqlPattern.indexOf('%', 1) == sqlPattern.length() - 1) { + String mid = sqlPattern.substring(1, sqlPattern.length() - 1); + return builder.contains(builder.indexOf(fieldRefExpr.getName()), mid); + } + } + } + } + + // TODO is_xxx, between_xxx, similar, in, not_in, not? + + throw new UnsupportedExpression(); + } + + private Predicate visitBiFunction( + List children, + BiFunction visit1, + BiFunction visit2) { + Optional fieldRefExpr = extractFieldReference(children.get(0)); + if (fieldRefExpr.isPresent() && builder.indexOf(fieldRefExpr.get().getName()) != -1) { + Object literal = + extractLiteral(fieldRefExpr.get().getOutputDataType(), children.get(1)); + return visit1.apply(builder.indexOf(fieldRefExpr.get().getName()), literal); + } else { + fieldRefExpr = extractFieldReference(children.get(1)); + if (fieldRefExpr.isPresent()) { + Object literal = + extractLiteral(fieldRefExpr.get().getOutputDataType(), children.get(0)); + return visit2.apply(builder.indexOf(fieldRefExpr.get().getName()), literal); + } + } + + throw new UnsupportedExpression(); + } + + private Optional extractFieldReference(Expression expression) { + if (expression instanceof FieldReferenceExpression) { + return Optional.of((FieldReferenceExpression) expression); + } + return Optional.empty(); + } + + private Object extractLiteral(DataType expectedType, Expression expression) { + LogicalType expectedLogicalType = expectedType.getLogicalType(); + if (!supportsPredicate(expectedLogicalType)) { + throw new UnsupportedExpression(); + } + + if (expression instanceof ValueLiteralExpression) { + ValueLiteralExpression valueExpression = (ValueLiteralExpression) expression; + if (valueExpression.isNull()) { + return null; + } + + DataType actualType = valueExpression.getOutputDataType(); + LogicalType actualLogicalType = actualType.getLogicalType(); + Optional valueOpt = valueExpression.getValueAs(actualType.getConversionClass()); + if (valueOpt.isPresent()) { + Object value = valueOpt.get(); + if (actualLogicalType.getTypeRoot().equals(expectedLogicalType.getTypeRoot()) + && !isStringType(expectedLogicalType)) { + return FlinkAsFlussRow.fromFlinkObject(value, expectedType); + } else if (supportsImplicitCast(actualLogicalType, expectedLogicalType)) { + try { + return TypeUtils.castFromString( + value.toString(), FlinkConversions.toFlussType(expectedType)); + } catch (Exception ignored) { + } + } + } + } + + throw new UnsupportedExpression(); + } + + private boolean isStringType(LogicalType type) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + return true; + default: + return false; + } + } + + private boolean supportsPredicate(LogicalType type) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BOOLEAN: + case BINARY: + case VARBINARY: + case DECIMAL: + case TINYINT: + case SMALLINT: + case INTEGER: + case BIGINT: + case FLOAT: + case DOUBLE: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + case INTERVAL_YEAR_MONTH: + case INTERVAL_DAY_TIME: + return true; + default: + return false; + } + } + + @Override + public Predicate visit(ValueLiteralExpression valueLiteralExpression) { + throw new UnsupportedExpression(); + } + + @Override + public Predicate visit(FieldReferenceExpression fieldReferenceExpression) { + throw new UnsupportedExpression(); + } + + @Override + public Predicate visit(TypeLiteralExpression typeLiteralExpression) { + throw new UnsupportedExpression(); + } + + @Override + public Predicate visit(Expression expression) { + throw new UnsupportedExpression(); + } + + /** + * Try best to convert a {@link ResolvedExpression} to {@link Predicate}. + * + * @param filter a resolved expression + * @return {@link Predicate} if no {@link UnsupportedExpression} thrown. + */ + public static Optional convert(RowType rowType, ResolvedExpression filter) { + try { + return Optional.ofNullable(filter.accept(new PredicateConverter(rowType))); + } catch (UnsupportedExpression e) { + return Optional.empty(); + } + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java index b620d1d9f7..f97187c14f 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumerator.java @@ -17,33 +17,35 @@ package org.apache.fluss.flink.source.enumerator; -import com.alibaba.fluss.client.Connection; -import com.alibaba.fluss.client.ConnectionFactory; -import com.alibaba.fluss.client.admin.Admin; -import com.alibaba.fluss.client.metadata.KvSnapshots; -import com.alibaba.fluss.config.ConfigOptions; -import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.flink.lakehouse.LakeSplitGenerator; -import com.alibaba.fluss.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; -import com.alibaba.fluss.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; -import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer; -import com.alibaba.fluss.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; -import com.alibaba.fluss.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; -import com.alibaba.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; -import com.alibaba.fluss.flink.source.event.PartitionsRemovedEvent; -import com.alibaba.fluss.flink.source.split.HybridSnapshotLogSplit; -import com.alibaba.fluss.flink.source.split.LogSplit; -import com.alibaba.fluss.flink.source.split.SourceSplitBase; -import com.alibaba.fluss.flink.source.state.SourceEnumeratorState; -import com.alibaba.fluss.metadata.PartitionInfo; -import com.alibaba.fluss.metadata.TableBucket; -import com.alibaba.fluss.metadata.TableInfo; -import com.alibaba.fluss.metadata.TablePath; -import com.alibaba.fluss.predicate.Predicate; -import com.alibaba.fluss.row.BinaryString; -import com.alibaba.fluss.row.GenericRow; -import com.alibaba.fluss.row.InternalRow; -import com.alibaba.fluss.utils.ExceptionUtils; +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.metadata.KvSnapshots; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.lake.LakeSplitGenerator; +import org.apache.fluss.flink.source.enumerator.initializer.BucketOffsetsRetrieverImpl; +import org.apache.fluss.flink.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer; +import org.apache.fluss.flink.source.enumerator.initializer.OffsetsInitializer.BucketOffsetsRetriever; +import org.apache.fluss.flink.source.enumerator.initializer.SnapshotOffsetsInitializer; +import org.apache.fluss.flink.source.event.PartitionBucketsUnsubscribedEvent; +import org.apache.fluss.flink.source.event.PartitionsRemovedEvent; +import org.apache.fluss.flink.source.split.HybridSnapshotLogSplit; +import org.apache.fluss.flink.source.split.LogSplit; +import org.apache.fluss.flink.source.split.SourceSplitBase; +import org.apache.fluss.flink.source.state.SourceEnumeratorState; +import org.apache.fluss.lake.source.LakeSource; +import org.apache.fluss.lake.source.LakeSplit; +import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.predicate.Predicate; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.ExceptionUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.source.SourceEvent; @@ -131,7 +133,11 @@ public class FlinkSourceEnumerator private boolean lakeEnabled = false; - private Predicate predicate; + private volatile boolean closed = false; + + private Predicate partitionFilters; + + @Nullable private final LakeSource lakeSource; public FlinkSourceEnumerator( TablePath tablePath, @@ -142,20 +148,18 @@ public FlinkSourceEnumerator( OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, boolean streaming, - Predicate predicate) { + Predicate partitionFilters) { this( tablePath, flussConf, hasPrimaryKey, isPartitioned, context, - Collections.emptySet(), - Collections.emptyMap(), - null, startingOffsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - predicate); + partitionFilters, + null); } public FlinkSourceEnumerator( @@ -164,24 +168,25 @@ public FlinkSourceEnumerator( boolean hasPrimaryKey, boolean isPartitioned, SplitEnumeratorContext context, - Set assignedTableBuckets, - Map assignedPartitions, - List pendingHybridLakeFlussSplits, OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, - boolean streaming) { + boolean streaming, + Predicate partitionFilters, + @Nullable LakeSource lakeSource) { this( tablePath, flussConf, - isPartitioned, hasPrimaryKey, + isPartitioned, context, - assignedTableBuckets, - assignedPartitions, + Collections.emptySet(), + Collections.emptyMap(), + null, startingOffsetsInitializer, scanPartitionDiscoveryIntervalMs, streaming, - null); + partitionFilters, + lakeSource); } public FlinkSourceEnumerator( @@ -192,10 +197,12 @@ public FlinkSourceEnumerator( SplitEnumeratorContext context, Set assignedTableBuckets, Map assignedPartitions, + List pendingHybridLakeFlussSplits, OffsetsInitializer startingOffsetsInitializer, long scanPartitionDiscoveryIntervalMs, boolean streaming, - Predicate predicate) { + Predicate partitionFilters, + @Nullable LakeSource lakeSource) { this.tablePath = checkNotNull(tablePath); this.flussConf = checkNotNull(flussConf); this.hasPrimaryKey = hasPrimaryKey; @@ -211,9 +218,10 @@ public FlinkSourceEnumerator( : new LinkedList<>(pendingHybridLakeFlussSplits); this.scanPartitionDiscoveryIntervalMs = scanPartitionDiscoveryIntervalMs; this.streaming = streaming; + this.partitionFilters = partitionFilters; this.stoppingOffsetsInitializer = streaming ? new NoStoppingOffsetsInitializer() : OffsetsInitializer.latest(); - this.predicate = predicate; + this.lakeSource = lakeSource; } @Override @@ -332,11 +340,13 @@ private List initNonPartitionedSplits() { } private Set listPartitions() { - + if (closed) { + return Collections.emptySet(); + } try { List partitionInfos = flussAdmin.listPartitionInfos(tablePath).get(); partitionInfos = applyPartitionFilter(partitionInfos); - return new HashSet<>(partitionInfos); + return new LinkedHashSet<>(partitionInfos); } catch (Exception e) { throw new FlinkRuntimeException( String.format("Failed to list partitions for %s", tablePath), @@ -346,22 +356,37 @@ private Set listPartitions() { /** Apply partition filter. */ private List applyPartitionFilter(List partitionInfos) { - if (predicate == null) { + if (partitionFilters == null) { return partitionInfos; } else { + int originalSize = partitionInfos.size(); List filteredPartitionInfos = partitionInfos.stream() .filter( partitionInfo -> - predicate.test( + partitionFilters.test( convertPartitionInfoToInternalRow( partitionInfo))) .collect(Collectors.toList()); - LOG.info( - "Filtered partitions {} for table {} with predicate: {}", - filteredPartitionInfos, - tablePath, - predicate); + + int filteredSize = filteredPartitionInfos.size(); + // Only log when there's actual filtering happening or when it's the first time + if (originalSize != filteredSize) { + LOG.info( + "Applied partition filter for table {}: {} partitions filtered to {} partitions with predicate: {}", + tablePath, + originalSize, + filteredSize, + partitionFilters); + if (LOG.isDebugEnabled()) { + LOG.debug("Filtered partitions: {}", filteredPartitionInfos); + } + } else if (LOG.isDebugEnabled()) { + LOG.debug( + "Partition filter applied for table {} but no partitions were filtered out (total: {})", + tablePath, + originalSize); + } return filteredPartitionInfos; } } @@ -386,17 +411,43 @@ private void checkPartitionChanges(Set partitionInfos, Throwable LOG.error("Failed to list partitions for {}", tablePath, t); return; } + + if (LOG.isDebugEnabled()) { + LOG.debug( + "Checking partition changes for table {}, found {} partitions", + tablePath, + partitionInfos.size()); + } + final PartitionChange partitionChange = getPartitionChange(partitionInfos); if (partitionChange.isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("No partition changes detected for table {}", tablePath); + } return; } // handle removed partitions - handlePartitionsRemoved(partitionChange.removedPartitions); + if (!partitionChange.removedPartitions.isEmpty()) { + LOG.info( + "Handling {} removed partitions for table {}: {}", + partitionChange.removedPartitions.size(), + tablePath, + partitionChange.removedPartitions); + handlePartitionsRemoved(partitionChange.removedPartitions); + } // handle new partitions - context.callAsync( - () -> initPartitionedSplits(partitionChange.newPartitions), this::handleSplitsAdd); + if (!partitionChange.newPartitions.isEmpty()) { + LOG.info( + "Handling {} new partitions for table {}: {}", + partitionChange.newPartitions.size(), + tablePath, + partitionChange.newPartitions); + context.callAsync( + () -> initPartitionedSplits(partitionChange.newPartitions), + this::handleSplitsAdd); + } } private PartitionChange getPartitionChange(Set fetchedPartitionInfos) { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java index 3de0a680b7..04b43693f1 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceITCase.java @@ -1024,7 +1024,7 @@ void testStreamingReadMultiPartitionPushDown() throws Exception { .stream() .filter(s -> s.contains("2025")) .collect(Collectors.toList()); - waitUtilAllBucketFinishSnapshot( + waitUntilAllBucketFinishSnapshot( admin, tablePath, Arrays.asList("2025$1", "2025$2", "2026$1")); String plan = tEnv.explainSql("select * from multi_partitioned_table where c ='2025'"); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index b4ba22deae..5dc4af4708 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -357,6 +357,7 @@ void testRestore() throws Throwable { OffsetsInitializer.earliest(), DEFAULT_SCAN_PARTITION_DISCOVERY_INTERVAL_MS, streaming, + null, null); enumerator.start(); diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index a0a3b8e8c3..aaa08c07db 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -368,13 +368,54 @@ org.apache.fluss.metrics.* - - com.alibaba.fluss.lakehouse.* - com.alibaba.fluss.connector.flink.lakehouse.* - - com.alibaba.fluss.lakehouse.cli.* + org.apache.fluss.flink.lake.* + org.apache.fluss.kafka.* + + org.apache.fluss.tools.ci.* - com.alibaba.fluss.predicate.* + org.apache.fluss.predicate.* + + org.apache.fluss.lake.source.* + + org.apache.fluss.dist.DummyClass + org.apache.fluss.flink.DummyClass120 + org.apache.fluss.lake.batch.ArrowRecordBatch + org.apache.fluss.lake.committer.CommittedLakeSnapshot + org.apache.fluss.lake.paimon.FlussDataTypeToPaimonDataType + + org.apache.fluss.lake.lance.* + + org.apache.fluss.lake.iceberg.* + org.apache.fluss.row.encode.iceberg.* + org.apache.fluss.bucketing.IcebergBucketingFunction + + org.apache.fluss.flink.tiering.source.TieringSourceOptions + org.apache.fluss.flink.tiering.source.TieringSource.Builder + org.apache.fluss.flink.tiering.source.TieringSource + + org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator + + + org.apache.fluss.flink.tiering.source.enumerator.TieringSourceEnumerator.HeartBeatHelper + + org.apache.fluss.flink.tiering.source.TieringWriterInitContext + + org.apache.fluss.flink.tiering.source.TieringSourceReader + org.apache.fluss.flink.tiering.source.TableBucketWriteResultEmitter + + + org.apache.fluss.flink.tiering.source.TableBucketWriteResultTypeInfo* + + + org.apache.fluss.flink.tiering.committer.TieringCommitOperatorFactory + + org.apache.fluss.flink.tiering.committer.CommittableMessageTypeInfo* + + + org.apache.fluss.flink.tiering.LakeTieringJobBuilder + + org.apache.fluss.flink.tiering.FlussLakeTieringEntrypoint +