godfreyhe commented on code in PR #20374: URL: https://github.com/apache/flink/pull/20374#discussion_r937308705
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/dynamicfiltering/DynamicFilteringDataCollectorOperator.java: ########## @@ -0,0 +1,183 @@ +/* + * 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.flink.table.runtime.operators.dynamicfiltering; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArrayComparator; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.source.event.SourceEventWrapper; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.connector.source.DynamicFilteringData; +import org.apache.flink.table.connector.source.DynamicFilteringEvent; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Operator to collect and build the {@link DynamicFilteringData} for sources that supports dynamic + * filtering. + */ +public class DynamicFilteringDataCollectorOperator extends AbstractStreamOperator<Object> + implements OneInputStreamOperator<RowData, Object> { + + private static final Logger LOG = + LoggerFactory.getLogger(DynamicFilteringDataCollectorOperator.class); + + private final RowType dynamicFilteringFieldType; + private final List<Integer> dynamicFilteringFieldIndices; + private final long threshold; + private final OperatorEventGateway operatorEventGateway; + + private transient TypeInformation<RowData> typeInfo; + private transient TypeSerializer<RowData> serializer; + + private transient Set<byte[]> buffer; + private transient long currentSize; + + public DynamicFilteringDataCollectorOperator( + RowType dynamicFilteringFieldType, + List<Integer> dynamicFilteringFieldIndices, + long threshold, + OperatorEventGateway operatorEventGateway) { + this.dynamicFilteringFieldType = checkNotNull(dynamicFilteringFieldType); + this.dynamicFilteringFieldIndices = checkNotNull(dynamicFilteringFieldIndices); + this.threshold = threshold; + this.operatorEventGateway = checkNotNull(operatorEventGateway); + } + + @Override + public void open() throws Exception { + super.open(); + this.typeInfo = InternalTypeInfo.of(dynamicFilteringFieldType); + this.serializer = typeInfo.createSerializer(new ExecutionConfig()); + this.buffer = new TreeSet<>(new BytePrimitiveArrayComparator(true)::compare); + this.currentSize = 0L; + } + + @Override + public void processElement(StreamRecord<RowData> element) throws Exception { + if (exceedThreshold()) { + return; + } + + RowData value = element.getValue(); + GenericRowData rowData = new GenericRowData(dynamicFilteringFieldIndices.size()); + for (int i = 0; i < dynamicFilteringFieldIndices.size(); ++i) { + LogicalType type = dynamicFilteringFieldType.getTypeAt(i); + int index = dynamicFilteringFieldIndices.get(i); + switch (type.getTypeRoot()) { + case INTEGER: + rowData.setField(i, value.getInt(index)); + break; + case BIGINT: + rowData.setField(i, value.getLong(index)); + break; + case VARCHAR: + rowData.setField(i, value.getString(index)); + break; + default: Review Comment: we should more types, such as: CHAR, SMALLINT, TINYINT, etc. It's better we can defined the supported types in `DynamicFilteringData` , and then DynamicPartitionPruningRule can skip the query which contains unsupported types. ########## flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/dynamicfiltering/DynamicFilteringDataCollectorOperatorTest.java: ########## @@ -0,0 +1,123 @@ +/* + * 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.flink.table.runtime.operators.dynamicfiltering; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.source.event.SourceEventWrapper; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.connector.source.DynamicFilteringData; +import org.apache.flink.table.connector.source.DynamicFilteringEvent; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link DynamicFilteringDataCollectorOperator}. */ +class DynamicFilteringDataCollectorOperatorTest { Review Comment: add some test for unsupported types ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/DynamicFilteringData.java: ########## @@ -0,0 +1,184 @@ +/* + * 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.flink.table.connector.source; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArrayComparator; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.io.ByteArrayInputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Data for dynamic filtering. */ +public class DynamicFilteringData implements Serializable { + private final TypeInformation<RowData> typeInfo; + private final RowType rowType; + + /** The list should be sorted and distinct. */ + private final List<byte[]> serializedData; + + private final boolean isFiltering; Review Comment: please add some comments about this field ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/DynamicFilteringData.java: ########## @@ -0,0 +1,184 @@ +/* + * 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.flink.table.connector.source; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArrayComparator; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.io.ByteArrayInputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Data for dynamic filtering. */ +public class DynamicFilteringData implements Serializable { + private final TypeInformation<RowData> typeInfo; + private final RowType rowType; + + /** The list should be sorted and distinct. */ + private final List<byte[]> serializedData; + + private final boolean isFiltering; + + private transient volatile boolean prepared = false; + private transient Map<Integer, List<RowData>> dataMap; + private transient RowData.FieldGetter[] fieldGetters; + + public DynamicFilteringData( + TypeInformation<RowData> typeInfo, + RowType rowType, + List<byte[]> serializedData, + boolean isFiltering) { + this.typeInfo = checkNotNull(typeInfo); + this.rowType = checkNotNull(rowType); + this.serializedData = checkNotNull(serializedData); + this.isFiltering = isFiltering; + } + + public boolean isFiltering() { + return isFiltering; + } + + public RowType getRowType() { + return rowType; + } + + public boolean contains(RowData row) { + if (!isFiltering) { + return true; + } else if (row.getArity() != rowType.getFieldCount()) { + throw new TableException("The arity of RowData is different"); + } else { + prepare(); + List<RowData> mayMatchRowData = dataMap.get(hash(row)); + if (mayMatchRowData == null) { + return false; + } + for (RowData mayMatch : mayMatchRowData) { + if (matchRow(row, mayMatch)) { + return true; + } + } + return false; + } + } + + private boolean matchRow(RowData row, RowData mayMatch) { + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!Objects.equals( + fieldGetters[i].getFieldOrNull(row), + fieldGetters[i].getFieldOrNull(mayMatch))) { + return false; + } + } + return true; + } + + private void prepare() { + if (!prepared) { + synchronized (this) { + if (!prepared) { + doPrepare(); + prepared = true; + } + } + } + } + + private void doPrepare() { + this.dataMap = new HashMap<>(); + if (isFiltering) { + this.fieldGetters = + IntStream.range(0, rowType.getFieldCount()) + .mapToObj(i -> RowData.createFieldGetter(rowType.getTypeAt(i), i)) + .toArray(RowData.FieldGetter[]::new); + + TypeSerializer<RowData> serializer = typeInfo.createSerializer(new ExecutionConfig()); + for (byte[] bytes : serializedData) { + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(bais)) { + RowData partition = serializer.deserialize(inView); + List<RowData> partitions = + dataMap.computeIfAbsent(hash(partition), k -> new ArrayList<>()); + partitions.add(partition); + } catch (Exception e) { + throw new TableException("Unable to deserialize the value.", e); + } + } + } + } + + private int hash(RowData row) { + return Objects.hash(Arrays.stream(fieldGetters).map(g -> g.getFieldOrNull(row)).toArray()); + } + + public static boolean isEqual(DynamicFilteringData data, DynamicFilteringData another) { Review Comment: implements `equals` method directly ? ########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/DynamicFilteringData.java: ########## @@ -0,0 +1,184 @@ +/* + * 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.flink.table.connector.source; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArrayComparator; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.io.ByteArrayInputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Data for dynamic filtering. */ +public class DynamicFilteringData implements Serializable { + private final TypeInformation<RowData> typeInfo; + private final RowType rowType; + + /** The list should be sorted and distinct. */ + private final List<byte[]> serializedData; + + private final boolean isFiltering; + + private transient volatile boolean prepared = false; + private transient Map<Integer, List<RowData>> dataMap; + private transient RowData.FieldGetter[] fieldGetters; + + public DynamicFilteringData( + TypeInformation<RowData> typeInfo, + RowType rowType, + List<byte[]> serializedData, + boolean isFiltering) { + this.typeInfo = checkNotNull(typeInfo); + this.rowType = checkNotNull(rowType); + this.serializedData = checkNotNull(serializedData); + this.isFiltering = isFiltering; + } + + public boolean isFiltering() { + return isFiltering; + } + + public RowType getRowType() { + return rowType; + } + + public boolean contains(RowData row) { + if (!isFiltering) { + return true; + } else if (row.getArity() != rowType.getFieldCount()) { + throw new TableException("The arity of RowData is different"); + } else { + prepare(); + List<RowData> mayMatchRowData = dataMap.get(hash(row)); + if (mayMatchRowData == null) { + return false; + } + for (RowData mayMatch : mayMatchRowData) { + if (matchRow(row, mayMatch)) { + return true; + } + } + return false; + } + } + + private boolean matchRow(RowData row, RowData mayMatch) { + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!Objects.equals( + fieldGetters[i].getFieldOrNull(row), + fieldGetters[i].getFieldOrNull(mayMatch))) { + return false; + } + } + return true; + } + + private void prepare() { + if (!prepared) { + synchronized (this) { + if (!prepared) { + doPrepare(); + prepared = true; + } + } + } + } + + private void doPrepare() { + this.dataMap = new HashMap<>(); + if (isFiltering) { + this.fieldGetters = + IntStream.range(0, rowType.getFieldCount()) + .mapToObj(i -> RowData.createFieldGetter(rowType.getTypeAt(i), i)) + .toArray(RowData.FieldGetter[]::new); + + TypeSerializer<RowData> serializer = typeInfo.createSerializer(new ExecutionConfig()); + for (byte[] bytes : serializedData) { + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(bais)) { + RowData partition = serializer.deserialize(inView); + List<RowData> partitions = + dataMap.computeIfAbsent(hash(partition), k -> new ArrayList<>()); + partitions.add(partition); + } catch (Exception e) { + throw new TableException("Unable to deserialize the value.", e); + } + } + } + } + + private int hash(RowData row) { + return Objects.hash(Arrays.stream(fieldGetters).map(g -> g.getFieldOrNull(row)).toArray()); + } + + public static boolean isEqual(DynamicFilteringData data, DynamicFilteringData another) { + if (data == null) { + return another == null; + } + if (another == null + || (data.isFiltering != another.isFiltering) + || !data.typeInfo.equals(another.typeInfo) + || !data.rowType.equals(another.rowType) + || data.serializedData.size() != another.serializedData.size()) { + return false; + } + + BytePrimitiveArrayComparator comparator = new BytePrimitiveArrayComparator(true); + for (int i = 0; i < data.serializedData.size(); i++) { + if (comparator.compare(data.serializedData.get(i), another.serializedData.get(i)) + != 0) { + return false; + } + } + return true; + } + + @VisibleForTesting + public Collection<RowData> getData() { Review Comment: this method is not only used for testing, but also can be used for advanced filtering which `contains` methods can not meet the requirement ########## flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/dynamicfiltering/DynamicFilteringDataCollectorOperatorCoordinatorTest.java: ########## @@ -0,0 +1,74 @@ +/* + * 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.flink.table.runtime.operators.dynamicfiltering; + +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator; +import org.apache.flink.runtime.source.event.SourceEventWrapper; +import org.apache.flink.table.connector.source.DynamicFilteringData; +import org.apache.flink.table.connector.source.DynamicFilteringEvent; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link DynamicFilteringDataCollectorOperatorCoordinator}. */ +class DynamicFilteringDataCollectorOperatorCoordinatorTest { Review Comment: add some tests for failover case -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org