pltbkd commented on code in PR #20374:
URL: https://github.com/apache/flink/pull/20374#discussion_r937316474


##########
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:
   As the data struct may contain large scale of data, calculate the hashcode 
with all the data can be costing. So I suppose we'd better not to override the 
hashcode, as well as the equals.



##########
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:
   So far the method is not used in formal code, including the dynamic 
filtering implementation of hive source. I'd prefer to remain the annotation 
for now. We can remove it when it's actually used.



-- 
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

Reply via email to