zhuzhurk commented on code in PR #23004:
URL: https://github.com/apache/flink/pull/23004#discussion_r1269216986


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/runtimefilter/LocalRuntimeFilterBuilderOperator.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtimefilter;
+
+import org.apache.flink.runtime.operators.util.BloomFilter;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import 
org.apache.flink.table.runtime.operators.runtimefilter.util.RuntimeFilterUtils;
+import org.apache.flink.table.runtime.util.StreamRecordCollector;
+import org.apache.flink.util.Collector;
+
+import static 
org.apache.flink.table.runtime.operators.runtimefilter.util.RuntimeFilterUtils.OVER_MAX_ROW_COUNT;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Local runtime filter builder operator. */
+public class LocalRuntimeFilterBuilderOperator extends 
TableStreamOperator<RowData>
+        implements OneInputStreamOperator<RowData, RowData>, BoundedOneInput {

Review Comment:
   I's better add an empty line between the class declaration and its fields.



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/runtimefilter/GlobalRuntimeFilterBuilderOperatorTest.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.runtimefilter;
+
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.StopMode;
+import org.apache.flink.runtime.operators.util.BloomFilter;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness;
+import 
org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.operators.runtimefilter.util.RuntimeFilterUtils;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.IntType;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Queue;
+
+import static 
org.apache.flink.table.runtime.operators.runtimefilter.util.RuntimeFilterUtils.OVER_MAX_ROW_COUNT;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Test for {@link GlobalRuntimeFilterBuilderOperator}. */
+class GlobalRuntimeFilterBuilderOperatorTest {
+
+    @Test
+    void testNormalInputAndNormalOutput() throws Exception {
+        try (StreamTaskMailboxTestHarness<RowData> testHarness =
+                createGlobalRuntimeFilterBuilderOperatorHarness(10)) {
+            // process elements
+            testHarness.processElement(
+                    new StreamRecord<RowData>(
+                            GenericRowData.of(5, 
BloomFilter.toBytes(createBloomFilter1()))));
+            testHarness.processElement(
+                    new StreamRecord<RowData>(
+                            GenericRowData.of(5, 
BloomFilter.toBytes(createBloomFilter2()))));
+            testHarness.processEvent(new EndOfData(StopMode.DRAIN), 0);
+
+            // test the output
+            Queue<Object> outputs = testHarness.getOutput();
+            assertThat(outputs.size()).isEqualTo(1);
+
+            RowData outputRowData = ((StreamRecord<RowData>) 
outputs.poll()).getValue();
+            assertThat(outputRowData.getArity()).isEqualTo(2);
+
+            int globalCount = outputRowData.getInt(0);
+            BloomFilter globalBloomFilter = 
BloomFilter.fromBytes(outputRowData.getBinary(1));
+            assertThat(globalCount).isEqualTo(10);
+            assertThat(globalBloomFilter.testHash("var1".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var2".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var3".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var4".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var5".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var6".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var7".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var8".hashCode())).isTrue();
+            assertThat(globalBloomFilter.testHash("var9".hashCode())).isTrue();
+            
assertThat(globalBloomFilter.testHash("var10".hashCode())).isTrue();

Review Comment:
   Better to add some assertions for records that do not pass the test.



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/runtimefilter/LocalRuntimeFilterBuilderOperatorTest.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.runtimefilter;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.StopMode;
+import org.apache.flink.runtime.operators.util.BloomFilter;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness;
+import 
org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.data.writer.BinaryRowWriter;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.Serializable;
+import java.util.Queue;
+
+import static 
org.apache.flink.table.runtime.operators.runtimefilter.util.RuntimeFilterUtils.OVER_MAX_ROW_COUNT;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Test for {@link LocalRuntimeFilterBuilderOperator}. */
+class LocalRuntimeFilterBuilderOperatorTest implements Serializable {
+
+    @Test
+    void testNormalOutput() throws Exception {
+        // create test harness and process input elements
+        try (StreamTaskMailboxTestHarness<RowData> testHarness =
+                
createLocalRuntimeFilterBuilderOperatorHarnessAndProcessElements(5, 10)) {
+
+            // test the output bloom filter
+            Queue<Object> outputs = testHarness.getOutput();
+            assertThat(outputs.size()).isEqualTo(1);
+
+            RowData outputRowData = ((StreamRecord<RowData>) 
outputs.poll()).getValue();
+            assertThat(outputRowData.getArity()).isEqualTo(2);
+
+            int actualCount = outputRowData.getInt(0);
+            BloomFilter bloomFilter = 
BloomFilter.fromBytes(outputRowData.getBinary(1));
+            assertThat(actualCount).isEqualTo(5);
+            // test elements that should exist
+            assertThat(bloomFilterTestString(bloomFilter, "var1")).isTrue();
+            assertThat(bloomFilterTestString(bloomFilter, "var2")).isTrue();
+            assertThat(bloomFilterTestString(bloomFilter, "var3")).isTrue();
+            assertThat(bloomFilterTestString(bloomFilter, "var4")).isTrue();
+            assertThat(bloomFilterTestString(bloomFilter, "var5")).isTrue();
+            // test elements that should not exist
+            assertThat(bloomFilterTestString(bloomFilter, "var6")).isFalse();
+            assertThat(bloomFilterTestString(bloomFilter, "var7")).isFalse();
+            assertThat(bloomFilterTestString(bloomFilter, "var8")).isFalse();
+            assertThat(bloomFilterTestString(bloomFilter, "var9")).isFalse();
+            assertThat(bloomFilterTestString(bloomFilter, "var10")).isFalse();
+        }
+    }
+
+    /** Test the case that the output filter is over-max-row-count. */
+    @Test
+    void testOverMaxRowCountOutput() throws Exception {
+        // create test harness and process input elements
+        try (StreamTaskMailboxTestHarness<RowData> testHarness =
+                
createLocalRuntimeFilterBuilderOperatorHarnessAndProcessElements(3, 4)) {
+
+            // test the output bloom filter should be null
+            Queue<Object> outputs = testHarness.getOutput();
+            assertThat(outputs.size()).isEqualTo(1);
+
+            RowData outputRowData = ((StreamRecord<RowData>) 
outputs.poll()).getValue();
+            assertThat(outputRowData.getArity()).isEqualTo(2);
+
+            int actualCount = outputRowData.getInt(0);
+            assertThat(actualCount).isEqualTo(OVER_MAX_ROW_COUNT);
+            assertThat(outputRowData.isNullAt(1)).isTrue();
+        }
+    }
+
+    private static boolean bloomFilterTestString(BloomFilter bloomFilter, 
String string) {
+        final Projection<RowData, BinaryRowData> projection = new 
FirstStringFileldProjection();
+        return bloomFilter.testHash(
+                
projection.apply(GenericRowData.of(StringData.fromString(string))).hashCode());
+    }
+
+    private static StreamRecord<RowData> createRowDataRecord(String string, 
int integer) {
+        return new 
StreamRecord<>(GenericRowData.of(StringData.fromString(string), integer));
+    }
+
+    private static StreamTaskMailboxTestHarness<RowData>
+            createLocalRuntimeFilterBuilderOperatorHarnessAndProcessElements(
+                    int estimatedRowCount, int maxRowCount) throws Exception {
+        final GeneratedProjection buildProjectionCode =
+                new GeneratedProjection("", "", new Object[0]) {
+                    @Override
+                    public Projection newInstance(ClassLoader classLoader) {
+                        return new FirstStringFileldProjection();
+                    }
+                };
+
+        final TypeInformation<RowData> inputType =
+                InternalTypeInfo.ofFields(new VarCharType(), new IntType());
+        final LocalRuntimeFilterBuilderOperator operator =
+                new LocalRuntimeFilterBuilderOperator(
+                        buildProjectionCode, estimatedRowCount, maxRowCount);
+        StreamTaskMailboxTestHarness<RowData> testHarness =
+                new StreamTaskMailboxTestHarnessBuilder<>(
+                                OneInputStreamTask::new,
+                                InternalTypeInfo.ofFields(new IntType(), new 
BinaryType()))
+                        .setupOutputForSingletonOperatorChain(operator)
+                        .addInput(inputType)
+                        .build();
+
+        testHarness.processElement(createRowDataRecord("var1", 111));
+        testHarness.processElement(createRowDataRecord("var2", 222));
+        testHarness.processElement(createRowDataRecord("var3", 333));
+        testHarness.processElement(createRowDataRecord("var4", 444));
+        testHarness.processElement(createRowDataRecord("var5", 555));
+        testHarness.processEvent(new EndOfData(StopMode.DRAIN), 0);
+
+        return testHarness;
+    }
+
+    static final class FirstStringFileldProjection implements 
Projection<RowData, BinaryRowData> {

Review Comment:
   filed -> Field



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to