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


##########
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:
   I suppose we can add a static field descrbing all the supported types as a 
Set.
   Besides, since the setters of GenericRowData don't care the type, I suppose 
we can create FieldGetters for getting value from the input rowType, and 
directly set to the GenericRowData we create here in spite of the type. The 
code can be much cleaner and we can support more types easily. 



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