wanglijie95 commented on code in PR #22966:
URL: https://github.com/apache/flink/pull/22966#discussion_r1260516252


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/program/FlinkRuntimeFilterProgramTest.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.planner.plan.optimize.program;
+
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataLong;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.planner.factories.TestValuesCatalog;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+
+/** Test for {@link FlinkRuntimeFilterProgram}. */
+public class FlinkRuntimeFilterProgramTest extends TableTestBase {
+    // 128L * 1024L * 48 = 6MB
+    private static final long SUITABLE_DIM_ROW_COUNT = 128L * 1024L;
+    // 1024L * 1024L * 1024L * 60 = 60GB
+    private static final long SUITABLE_FACT_ROW_COUNT = 1024L * 1024L * 1024L;
+
+    private final BatchTableTestUtil util = 
batchTestUtil(TableConfig.getDefault());
+    private final TestValuesCatalog catalog =
+            new TestValuesCatalog("testCatalog", "test_database", true);
+
+    @Before
+    public void setup() {
+        catalog.open();
+        util.tableEnv().registerCatalog("testCatalog", catalog);
+        util.tableEnv().useCatalog("testCatalog");
+        TableConfig tableConfig = util.tableEnv().getConfig();
+        
tableConfig.set(OptimizerConfigOptions.TABLE_OPTIMIZER_RUNTIME_FILTER_ENABLED, 
true);
+        tableConfig.set(
+                
OptimizerConfigOptions.TABLE_OPTIMIZER_RUNTIME_FILTER_MAX_BUILD_DATA_SIZE,
+                MemorySize.parse("10m"));
+        tableConfig.set(
+                
OptimizerConfigOptions.TABLE_OPTIMIZER_RUNTIME_FILTER_MIN_PROBE_DATA_SIZE,
+                MemorySize.parse("10g"));
+        tableConfig.set(
+                
OptimizerConfigOptions.TABLE_OPTIMIZER_RUNTIME_FILTER_MIN_FILTER_RATIO, 0.5);

Review Comment:
   I explicitly set the runtime-filter related config options here, due to the 
default value may be changed later(The best default values are still in 
testing). We explicitly set them here, even if the default values are changed 
later, current test will not be affected.



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