You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/01 03:30:57 UTC
[flink] 02/02: [FLINK-13375][table] Improve config names in
ExecutionConfigOptions and OptimizerConfigOptions
This is an automated email from the ASF dual-hosted git repository.
jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 796cf8d8997fdc2b97faab8a4fc0d0f134280af9
Author: Jark Wu <im...@gmail.com>
AuthorDate: Sat Jul 27 13:54:34 2019 +0800
[FLINK-13375][table] Improve config names in ExecutionConfigOptions and OptimizerConfigOptions
This closes #9203
---
.../flink/table/catalog/hive/HiveTestUtils.java | 4 +-
.../table/api/config/ExecutionConfigOptions.java | 242 ++++++++++-----------
.../table/api/config/OptimizerConfigOptions.java | 95 ++++----
.../table/planner/delegation/BatchExecutor.java | 4 +-
.../plan/nodes/resource/NodeResourceUtil.java | 6 +-
.../table/planner/plan/utils/OperatorType.java | 2 +-
.../planner/utils/AggregatePhaseStrategy.java | 2 +-
.../table/planner/utils/TableConfigUtils.java | 11 +-
.../codegen/NestedLoopJoinCodeGenerator.scala | 8 +-
.../plan/metadata/FlinkRelMdColumnNullCount.scala | 2 +-
.../plan/metadata/FlinkRelMdDistinctRowCount.scala | 2 +-
.../plan/metadata/FlinkRelMdDistribution.scala | 2 +-
.../planner/plan/metadata/FlinkRelMdRowCount.scala | 6 +-
.../plan/metadata/SelectivityEstimator.scala | 2 +-
.../plan/nodes/common/CommonLookupJoin.scala | 4 +-
.../plan/nodes/logical/FlinkLogicalSort.scala | 8 +-
.../nodes/physical/batch/BatchExecExchange.scala | 2 +-
.../physical/batch/BatchExecHashAggregate.scala | 2 +-
.../batch/BatchExecHashAggregateBase.scala | 7 +-
.../nodes/physical/batch/BatchExecHashJoin.scala | 7 +-
.../batch/BatchExecHashWindowAggregateBase.scala | 9 +-
.../physical/batch/BatchExecNestedLoopJoin.scala | 7 +-
.../physical/batch/BatchExecOverAggregate.scala | 9 +-
.../plan/nodes/physical/batch/BatchExecRank.scala | 2 +-
.../plan/nodes/physical/batch/BatchExecSort.scala | 7 +-
.../physical/batch/BatchExecSortAggregate.scala | 2 +-
.../physical/batch/BatchExecSortMergeJoin.scala | 12 +-
.../batch/BatchExecSortWindowAggregateBase.scala | 2 +-
.../physical/stream/StreamExecDeduplicate.scala | 2 +-
.../stream/StreamExecGlobalGroupAggregate.scala | 2 +-
.../physical/stream/StreamExecGroupAggregate.scala | 2 +-
.../nodes/physical/stream/StreamExecLimit.scala | 3 +-
.../nodes/physical/stream/StreamExecRank.scala | 6 +-
.../nodes/physical/stream/StreamExecSort.scala | 6 +-
.../physical/stream/StreamExecSortLimit.scala | 2 +-
.../stream/StreamExecWatermarkAssigner.scala | 4 +-
.../table/planner/plan/optimize/RelNodeBlock.scala | 14 +-
.../StreamCommonSubGraphBasedOptimizer.scala | 4 +-
.../plan/optimize/program/FlinkBatchProgram.scala | 2 +-
.../plan/optimize/program/FlinkStreamProgram.scala | 2 +-
.../table/planner/plan/reuse/SubplanReuser.scala | 4 +-
.../rules/logical/JoinDeriveNullFilterRule.scala | 6 +-
.../PushFilterIntoTableSourceScanRule.scala | 2 +-
.../plan/rules/logical/SplitAggregateRule.scala | 12 +-
.../physical/batch/BatchExecHashAggRule.scala | 6 +-
.../physical/batch/BatchExecHashJoinRule.scala | 4 +-
.../physical/batch/BatchExecJoinRuleBase.scala | 10 +-
.../physical/batch/BatchExecSortAggRule.scala | 6 +-
.../batch/BatchExecSortMergeJoinRule.scala | 10 +-
.../rules/physical/batch/BatchExecSortRule.scala | 6 +-
.../batch/BatchExecWindowAggregateRule.scala | 6 +-
.../physical/stream/IncrementalAggregateRule.scala | 8 +-
.../stream/MiniBatchIntervalInferRule.scala | 4 +-
.../stream/TwoStageOptimizedAggregateRule.scala | 2 +-
.../table/planner/plan/utils/AggregateUtil.scala | 5 +-
.../table/planner/plan/utils/FlinkRelOptUtil.scala | 2 +-
.../table/planner/plan/utils/FlinkRexUtil.scala | 4 +-
.../flink/table/planner/plan/utils/RankUtil.scala | 2 +-
.../planner/plan/utils/WindowEmitStrategy.scala | 24 +-
.../table/planner/plan/utils/WindowJoinUtil.scala | 2 +-
.../ShuffleStageParallelismCalculatorTest.java | 6 +-
.../apache/flink/table/api/batch/ExplainTest.scala | 2 +-
.../flink/table/api/stream/ExplainTest.scala | 4 +-
.../table/planner/catalog/CatalogTableITCase.scala | 2 +-
.../plan/batch/sql/DagOptimizationTest.scala | 24 +-
.../plan/batch/sql/DeadlockBreakupTest.scala | 46 ++--
.../plan/batch/sql/RemoveCollationTest.scala | 40 ++--
.../planner/plan/batch/sql/RemoveShuffleTest.scala | 114 +++++-----
.../planner/plan/batch/sql/SetOperatorsTest.scala | 2 +-
.../table/planner/plan/batch/sql/SinkTest.scala | 2 +-
.../planner/plan/batch/sql/SortLimitTest.scala | 24 +-
.../table/planner/plan/batch/sql/SortTest.scala | 22 +-
.../planner/plan/batch/sql/SubplanReuseTest.scala | 46 ++--
.../plan/batch/sql/agg/HashAggregateTest.scala | 4 +-
.../plan/batch/sql/agg/SortAggregateTest.scala | 4 +-
.../plan/batch/sql/agg/WindowAggregateTest.scala | 2 +-
.../batch/sql/join/BroadcastHashJoinTest.scala | 4 +-
.../sql/join/BroadcastHashSemiAntiJoinTest.scala | 4 +-
.../plan/batch/sql/join/LookupJoinTest.scala | 2 +-
.../plan/batch/sql/join/NestedLoopJoinTest.scala | 2 +-
.../sql/join/NestedLoopSemiAntiJoinTest.scala | 2 +-
.../plan/batch/sql/join/ShuffledHashJoinTest.scala | 2 +-
.../sql/join/ShuffledHashSemiAntiJoinTest.scala | 2 +-
.../plan/batch/sql/join/SortMergeJoinTest.scala | 2 +-
.../batch/sql/join/SortMergeSemiAntiJoinTest.scala | 2 +-
.../planner/plan/common/JoinReorderTestBase.scala | 2 +-
.../plan/nodes/resource/ExecNodeResourceTest.scala | 10 +-
.../logical/JoinDeriveNullFilterRuleTest.scala | 2 +-
.../rules/logical/SplitAggregateRuleTest.scala | 5 +-
.../RemoveRedundantLocalHashAggRuleTest.scala | 8 +-
.../RemoveRedundantLocalSortAggRuleTest.scala | 6 +-
.../RetractionRulesWithTwoStageAggTest.scala | 2 +-
.../plan/stream/sql/DagOptimizationTest.scala | 26 +--
.../stream/sql/MiniBatchIntervalInferTest.scala | 35 +--
.../plan/stream/sql/ModifiedMonotonicityTest.scala | 12 +-
.../planner/plan/stream/sql/SubplanReuseTest.scala | 14 +-
.../plan/stream/sql/agg/AggregateTest.scala | 16 +-
.../stream/sql/agg/DistinctAggregateTest.scala | 6 +-
.../stream/sql/agg/IncrementalAggregateTest.scala | 2 +-
.../stream/sql/agg/TwoStageAggregateTest.scala | 2 +-
.../plan/stream/table/TwoStageAggregateTest.scala | 8 +-
.../planner/runtime/batch/sql/CalcITCase.scala | 6 +-
.../planner/runtime/batch/sql/DecimalITCase.scala | 14 +-
.../planner/runtime/batch/sql/MiscITCase.scala | 4 +-
.../batch/sql/PartitionableSinkITCase.scala | 2 +-
.../runtime/batch/sql/SortLimitITCase.scala | 3 +-
.../planner/runtime/batch/sql/UnionITCase.scala | 6 +-
.../sql/agg/AggregateJoinTransposeITCase.scala | 4 +-
.../sql/agg/AggregateReduceGroupingITCase.scala | 20 +-
.../runtime/batch/sql/agg/HashAggITCase.scala | 2 +-
.../sql/agg/HashDistinctAggregateITCase.scala | 2 +-
.../runtime/batch/sql/agg/SortAggITCase.scala | 8 +-
.../sql/agg/SortDistinctAggregateITCase.scala | 2 +-
.../runtime/batch/sql/join/InnerJoinITCase.scala | 12 +-
.../runtime/batch/sql/join/JoinITCase.scala | 3 +-
.../runtime/batch/sql/join/JoinITCaseHelper.scala | 4 +-
.../runtime/batch/table/DecimalITCase.scala | 14 +-
.../planner/runtime/stream/sql/SortITCase.scala | 12 +-
.../runtime/stream/sql/SplitAggregateITCase.scala | 4 +-
.../runtime/stream/sql/WindowAggregateITCase.scala | 9 +-
.../planner/runtime/utils/BatchTestBase.scala | 16 +-
.../runtime/utils/StreamingWithAggTestBase.scala | 4 +-
.../utils/StreamingWithMiniBatchTestBase.scala | 10 +-
.../flink/table/planner/utils/TableTestBase.scala | 9 +-
.../runtime/hashtable/BaseHybridHashTable.java | 9 +-
.../operators/sort/BinaryExternalSorter.java | 12 +-
.../operators/sort/BufferedKVExternalSorter.java | 10 +-
.../runtime/hashtable/BinaryHashTableTest.java | 2 +-
.../table/runtime/hashtable/LongHashTableTest.java | 2 +-
.../operators/sort/BinaryExternalSorterTest.java | 8 +-
.../sort/BufferedKVExternalSorterTest.java | 4 +-
131 files changed, 693 insertions(+), 688 deletions(-)
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
index 04220c6..be9f112 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
@@ -34,7 +34,7 @@ import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.util.concurrent.ThreadLocalRandom;
-import static org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM;
+import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
/**
* Test utils for Hive connector.
@@ -103,7 +103,7 @@ public class HiveTestUtils {
public static TableEnvironment createTableEnv() {
EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build();
TableEnvironment tableEnv = TableEnvironment.create(settings);
- tableEnv.getConfig().getConfiguration().setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
+ tableEnv.getConfig().getConfiguration().setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
return tableEnv;
}
}
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
index 4a96b7a..544f028 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
@@ -32,94 +32,89 @@ public class ExecutionConfigOptions {
// ------------------------------------------------------------------------
// Source Options
// ------------------------------------------------------------------------
- public static final ConfigOption<String> SQL_EXEC_SOURCE_IDLE_TIMEOUT =
- key("sql.exec.source.idle.timeout")
- .defaultValue("-1 ms")
- .withDescription("When a source do not receive any elements for the timeout time, " +
- "it will be marked as temporarily idle. This allows downstream " +
- "tasks to advance their watermarks without the need to wait for " +
- "watermarks from this source while it is idle.");
+ public static final ConfigOption<String> TABLE_EXEC_SOURCE_IDLE_TIMEOUT =
+ key("table.exec.source.idle-timeout")
+ .defaultValue("-1 ms")
+ .withDescription("When a source do not receive any elements for the timeout time, " +
+ "it will be marked as temporarily idle. This allows downstream " +
+ "tasks to advance their watermarks without the need to wait for " +
+ "watermarks from this source while it is idle.");
// ------------------------------------------------------------------------
// Sort Options
// ------------------------------------------------------------------------
- public static final ConfigOption<Integer> SQL_EXEC_SORT_DEFAULT_LIMIT =
- key("sql.exec.sort.default.limit")
- .defaultValue(200)
- .withDescription("Default limit when user don't set a limit after order by. ");
+ public static final ConfigOption<Integer> TABLE_EXEC_SORT_DEFAULT_LIMIT =
+ key("table.exec.sort.default-limit")
+ .defaultValue(200)
+ .withDescription("Default limit when user don't set a limit after order by.");
- public static final ConfigOption<Integer> SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM =
- key("sql.exec.sort.max-num-file-handles")
- .defaultValue(128)
- .withDescription("The maximal fan-in for external merge sort. It limits the number of file handles per operator. " +
- "If it is too small, may cause intermediate merging. But if it is too large, " +
- "it will cause too many files opened at the same time, consume memory and lead to random reading.");
+ public static final ConfigOption<Integer> TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES =
+ key("table.exec.sort.max-num-file-handles")
+ .defaultValue(128)
+ .withDescription("The maximal fan-in for external merge sort. It limits the number of file handles per operator. " +
+ "If it is too small, may cause intermediate merging. But if it is too large, " +
+ "it will cause too many files opened at the same time, consume memory and lead to random reading.");
- public static final ConfigOption<Boolean> SQL_EXEC_SORT_ASYNC_MERGE_ENABLED =
- key("sql.exec.sort.async-merge.enabled")
- .defaultValue(true)
- .withDescription("Whether to asynchronously merge sorted spill files.");
+ public static final ConfigOption<Boolean> TABLE_EXEC_SORT_ASYNC_MERGE_ENABLED =
+ key("table.exec.sort.async-merge-enabled")
+ .defaultValue(true)
+ .withDescription("Whether to asynchronously merge sorted spill files.");
// ------------------------------------------------------------------------
// Spill Options
// ------------------------------------------------------------------------
- public static final ConfigOption<Boolean> SQL_EXEC_SPILL_COMPRESSION_ENABLED =
- key("sql.exec.spill.compression.enabled")
- .defaultValue(true)
- .withDescription("Whether to compress spilled data. " +
- "(Now include sort and hash agg and hash join)");
+ public static final ConfigOption<Boolean> TABLE_EXEC_SPILL_COMPRESSION_ENABLED =
+ key("table.exec.spill-compression.enabled")
+ .defaultValue(true)
+ .withDescription("Whether to compress spilled data. " +
+ "Currently we only support compress spilled data for sort and hash-agg and hash-join operators.");
- public static final ConfigOption<String> SQL_EXEC_SPILL_COMPRESSION_CODEC =
- key("sql.exec.spill.compression.codec")
- .defaultValue("lz4")
- .withDescription("Use that compression codec to compress spilled file. " +
- "Now we only support lz4.");
-
- public static final ConfigOption<Integer> SQL_EXEC_SPILL_COMPRESSION_BLOCK_SIZE =
- key("sql.exec.spill.compression.block-size")
- .defaultValue(64 * 1024)
- .withDescription("The buffer is to compress. The larger the buffer," +
- " the better the compression ratio, but the more memory consumption.");
+ public static final ConfigOption<String> TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE =
+ key("table.exec.spill-compression.block-size")
+ .defaultValue("64 kb")
+ .withDescription("The memory size used to do compress when spilling data. " +
+ "The larger the memory, the higher the compression ratio, " +
+ "but more memory resource will be consumed by the job.");
// ------------------------------------------------------------------------
// Resource Options
// ------------------------------------------------------------------------
- public static final ConfigOption<Integer> SQL_RESOURCE_DEFAULT_PARALLELISM =
- key("sql.resource.default.parallelism")
- .defaultValue(-1)
- .withDescription("Default parallelism of job operators. If it is <= 0, use parallelism of StreamExecutionEnvironment(" +
- "its default value is the num of cpu cores in the client host).");
+ public static final ConfigOption<Integer> TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM =
+ key("table.exec.resource.default-parallelism")
+ .defaultValue(-1)
+ .withDescription("Default parallelism of job operators. If it is <= 0, use parallelism of StreamExecutionEnvironment(" +
+ "its default value is the num of cpu cores in the client host).");
- public static final ConfigOption<Integer> SQL_RESOURCE_SOURCE_PARALLELISM =
- key("sql.resource.source.parallelism")
- .defaultValue(-1)
- .withDescription("Sets source parallelism, if it is <= 0, use " + SQL_RESOURCE_DEFAULT_PARALLELISM.key() + " to set source parallelism.");
+ public static final ConfigOption<Integer> TABLE_EXEC_RESOURCE_SOURCE_PARALLELISM =
+ key("table.exec.resource.source.parallelism")
+ .defaultValue(-1)
+ .withDescription("Sets source parallelism, if it is <= 0, use " + TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key() + " to set source parallelism.");
- public static final ConfigOption<Integer> SQL_RESOURCE_SINK_PARALLELISM =
- key("sql.resource.sink.parallelism")
- .defaultValue(-1)
- .withDescription("Sets sink parallelism, if it is <= 0, use " + SQL_RESOURCE_DEFAULT_PARALLELISM.key() + " to set sink parallelism.");
+ public static final ConfigOption<Integer> TABLE_EXEC_RESOURCE_SINK_PARALLELISM =
+ key("table.exec.resource.sink.parallelism")
+ .defaultValue(-1)
+ .withDescription("Sets sink parallelism, if it is <= 0, use " + TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key() + " to set sink parallelism.");
- public static final ConfigOption<Integer> SQL_RESOURCE_EXTERNAL_BUFFER_MEM =
- key("sql.resource.external-buffer.memory.mb")
- .defaultValue(10)
- .withDescription("Sets the externalBuffer memory size that is used in sortMergeJoin and overWindow.");
+ public static final ConfigOption<String> TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY =
+ key("table.exec.resource.external-buffer-memory")
+ .defaultValue("10 mb")
+ .withDescription("Sets the external buffer memory size that is used in sort merge join and nested join and over window.");
- public static final ConfigOption<Integer> SQL_RESOURCE_HASH_AGG_TABLE_MEM =
- key("sql.resource.hash-agg.table.memory.mb")
- .defaultValue(128)
- .withDescription("Sets the table memory size of hashAgg operator.");
+ public static final ConfigOption<String> TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY =
+ key("table.exec.resource.hash-agg.memory")
+ .defaultValue("128 mb")
+ .withDescription("Sets the managed memory size of hash aggregate operator.");
- public static final ConfigOption<Integer> SQL_RESOURCE_HASH_JOIN_TABLE_MEM =
- key("sql.resource.hash-join.table.memory.mb")
- .defaultValue(128)
- .withDescription("Sets the HashTable reserved memory for hashJoin operator. It defines the lower limit.");
+ public static final ConfigOption<String> TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY =
+ key("table.exec.resource.hash-join.memory")
+ .defaultValue("128 mb")
+ .withDescription("Sets the managed memory for hash join operator. It defines the lower limit.");
- public static final ConfigOption<Integer> SQL_RESOURCE_SORT_BUFFER_MEM =
- key("sql.resource.sort.buffer.memory.mb")
- .defaultValue(128)
- .withDescription("Sets the buffer memory size for sort.");
+ public static final ConfigOption<String> TABLE_EXEC_RESOURCE_SORT_MEMORY =
+ key("table.exec.resource.sort.memory")
+ .defaultValue("128 mb")
+ .withDescription("Sets the managed buffer memory size for sort operator.");
// ------------------------------------------------------------------------
// Agg Options
@@ -128,80 +123,71 @@ public class ExecutionConfigOptions {
/**
* See {@code org.apache.flink.table.runtime.operators.window.grouping.HeapWindowsGrouping}.
*/
- public static final ConfigOption<Integer> SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT =
- key("sql.exec.window-agg.buffer-size-limit")
- .defaultValue(100 * 1000)
- .withDescription("Sets the window elements buffer size limit used in group window agg operator.");
+ public static final ConfigOption<Integer> TABLE_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT =
+ key("table.exec.window-agg.buffer-size-limit")
+ .defaultValue(100 * 1000)
+ .withDescription("Sets the window elements buffer size limit used in group window agg operator.");
// ------------------------------------------------------------------------
// Async Lookup Options
// ------------------------------------------------------------------------
- public static final ConfigOption<Integer> SQL_EXEC_LOOKUP_ASYNC_BUFFER_CAPACITY =
- key("sql.exec.lookup.async.buffer-capacity")
- .defaultValue(100)
- .withDescription("The max number of async i/o operation that the async lookup join can trigger.");
+ public static final ConfigOption<Integer> TABLE_EXEC_ASYNC_LOOKUP_BUFFER_CAPACITY =
+ key("table.exec.async-lookup.buffer-capacity")
+ .defaultValue(100)
+ .withDescription("The max number of async i/o operation that the async lookup join can trigger.");
- public static final ConfigOption<String> SQL_EXEC_LOOKUP_ASYNC_TIMEOUT =
- key("sql.exec.lookup.async.timeout")
- .defaultValue("3 min")
- .withDescription("The async timeout for the asynchronous operation to complete.");
+ public static final ConfigOption<String> TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT =
+ key("table.exec.async-lookup.timeout")
+ .defaultValue("3 min")
+ .withDescription("The async timeout for the asynchronous operation to complete.");
// ------------------------------------------------------------------------
// MiniBatch Options
// ------------------------------------------------------------------------
- public static final ConfigOption<Boolean> SQL_EXEC_MINIBATCH_ENABLED =
- key("sql.exec.mini-batch.enabled")
- .defaultValue(false)
- .withDescription("Specifies whether to enable MiniBatch optimization. " +
- "MiniBatch is an optimization to buffer input records to reduce state access. " +
- "This is disabled by default. To enable this, users should set this config to true.");
-
- public static final ConfigOption<String> SQL_EXEC_MINIBATCH_ALLOW_LATENCY =
- key("sql.exec.mini-batch.allow-latency")
- .defaultValue("-1 ms")
- .withDescription("The maximum latency can be used for MiniBatch to buffer input records. " +
- "MiniBatch is an optimization to buffer input records to reduce state access. " +
- "MiniBatch is triggered with the allowed latency interval and when the maximum number of buffered records reached. " +
- "NOTE: If " + SQL_EXEC_MINIBATCH_ENABLED.key() + " is set true, its value must be greater than zero.");
-
- public static final ConfigOption<Long> SQL_EXEC_MINIBATCH_SIZE =
- key("sql.exec.mini-batch.size")
- .defaultValue(-1L)
- .withDescription("The maximum number of input records can be buffered for MiniBatch. " +
- "MiniBatch is an optimization to buffer input records to reduce state access. " +
- "MiniBatch is triggered with the allowed latency interval and when the maximum number of buffered records reached. " +
- "NOTE: MiniBatch only works for non-windowed aggregations currently. If " + SQL_EXEC_MINIBATCH_ENABLED.key() +
- " is set true, its value must be positive.");
-
- // ------------------------------------------------------------------------
- // State Options
- // ------------------------------------------------------------------------
- public static final ConfigOption<String> SQL_EXEC_STATE_TTL =
- key("sql.exec.state.ttl")
- .defaultValue("-1 ms")
- .withDescription("Specifies a minimum time interval for how long idle state " +
- "(i.e. state which was not updated), will be retained. State will never be " +
- "cleared until it was idle for less than the minimum time, and will be cleared " +
- "at some time after it was idle. Default is never clean-up the state.\n" +
- "NOTE: Cleaning up state requires additional overhead for bookkeeping.");
+ public static final ConfigOption<Boolean> TABLE_EXEC_MINIBATCH_ENABLED =
+ key("table.exec.mini-batch.enabled")
+ .defaultValue(false)
+ .withDescription("Specifies whether to enable MiniBatch optimization. " +
+ "MiniBatch is an optimization to buffer input records to reduce state access. " +
+ "This is disabled by default. To enable this, users should set this config to true. " +
+ "NOTE: If mini-batch is enabled, 'table.exec.mini-batch.allow-latency' and " +
+ "'table.exec.mini-batch.size' must be set.");
+
+ public static final ConfigOption<String> TABLE_EXEC_MINIBATCH_ALLOW_LATENCY =
+ key("table.exec.mini-batch.allow-latency")
+ .defaultValue("-1 ms")
+ .withDescription("The maximum latency can be used for MiniBatch to buffer input records. " +
+ "MiniBatch is an optimization to buffer input records to reduce state access. " +
+ "MiniBatch is triggered with the allowed latency interval and when the maximum number of buffered records reached. " +
+ "NOTE: If " + TABLE_EXEC_MINIBATCH_ENABLED.key() + " is set true, its value must be greater than zero.");
+
+ public static final ConfigOption<Long> TABLE_EXEC_MINIBATCH_SIZE =
+ key("table.exec.mini-batch.size")
+ .defaultValue(-1L)
+ .withDescription("The maximum number of input records can be buffered for MiniBatch. " +
+ "MiniBatch is an optimization to buffer input records to reduce state access. " +
+ "MiniBatch is triggered with the allowed latency interval and when the maximum number of buffered records reached. " +
+ "NOTE: MiniBatch only works for non-windowed aggregations currently. If " + TABLE_EXEC_MINIBATCH_ENABLED.key() +
+ " is set true, its value must be positive.");
// ------------------------------------------------------------------------
// Other Exec Options
// ------------------------------------------------------------------------
- public static final ConfigOption<String> SQL_EXEC_DISABLED_OPERATORS =
- key("sql.exec.disabled-operators")
- .defaultValue("")
- .withDescription("Mainly for testing. A comma-separated list of name of the OperatorType, each name " +
- "means a kind of disabled operator. Its default value is empty that means no operators are disabled. " +
- "If the configure's value is \"NestedLoopJoin, ShuffleHashJoin\", NestedLoopJoin and ShuffleHashJoin " +
- "are disabled. If configure's value is \"HashJoin\", ShuffleHashJoin and BroadcastHashJoin are disabled.");
-
- public static final ConfigOption<String> SQL_EXEC_SHUFFLE_MODE =
- key("sql.exec.shuffle-mode")
- .defaultValue("batch")
- .withDescription("Sets exec shuffle mode. Only batch or pipeline can be set.\n" +
- "batch: the job will run stage by stage. \n" +
- "pipeline: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when " +
- "the sender holds resource to wait to send data to the receiver.");
+ public static final ConfigOption<String> TABLE_EXEC_DISABLED_OPERATORS =
+ key("table.exec.disabled-operators")
+ .noDefaultValue()
+ .withDescription("Mainly for testing. A comma-separated list of operator names, each name " +
+ "represents a kind of disabled operator.\n" +
+ "Operators that can be disabled include \"NestedLoopJoin\", \"ShuffleHashJoin\", \"BroadcastHashJoin\", " +
+ "\"SortMergeJoin\", \"HashAgg\", \"SortAgg\".\n" +
+ "By default no operator is disabled.");
+
+ public static final ConfigOption<String> TABLE_EXEC_SHUFFLE_MODE =
+ key("table.exec.shuffle-mode")
+ .defaultValue("batch")
+ .withDescription("Sets exec shuffle mode. Only batch or pipeline can be set.\n" +
+ "batch: the job will run stage by stage. \n" +
+ "pipeline: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when " +
+ "the sender holds resource to wait to send data to the receiver.");
}
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
index 8be671a..8d6c1fd 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
@@ -32,59 +32,58 @@ public class OptimizerConfigOptions {
// ------------------------------------------------------------------------
// Optimizer Options
// ------------------------------------------------------------------------
- public static final ConfigOption<String> SQL_OPTIMIZER_AGG_PHASE_STRATEGY =
- key("sql.optimizer.agg-phase.strategy")
- .defaultValue("AUTO")
- .withDescription("Strategy for aggregate phase. Only AUTO, TWO_PHASE or ONE_PHASE can be set.\n" +
- "AUTO: No special enforcer for aggregate stage. Whether to choose two stage aggregate or one" +
- " stage aggregate depends on cost. \n" +
- "TWO_PHASE: Enforce to use two stage aggregate which has localAggregate and globalAggregate. " +
- "NOTE: If aggregate call does not support split into two phase, still use one stage aggregate.\n" +
- "ONE_PHASE: Enforce to use one stage aggregate which only has CompleteGlobalAggregate.");
+ public static final ConfigOption<String> TABLE_OPTIMIZER_AGG_PHASE_STRATEGY =
+ key("table.optimizer.agg-phase-strategy")
+ .defaultValue("AUTO")
+ .withDescription("Strategy for aggregate phase. Only AUTO, TWO_PHASE or ONE_PHASE can be set.\n" +
+ "AUTO: No special enforcer for aggregate stage. Whether to choose two stage aggregate or one" +
+ " stage aggregate depends on cost. \n" +
+ "TWO_PHASE: Enforce to use two stage aggregate which has localAggregate and globalAggregate. " +
+ "Note that if aggregate call does not support optimize into two phase, we will still use one stage aggregate.\n" +
+ "ONE_PHASE: Enforce to use one stage aggregate which only has CompleteGlobalAggregate.");
- public static final ConfigOption<Long> SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD =
- key("sql.optimizer.join.broadcast.threshold")
- .defaultValue(1024 * 1024L)
- .withDescription("Configures the maximum size in bytes for a table that will be broadcast to all worker " +
- "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. ");
+ public static final ConfigOption<Long> TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD =
+ key("table.optimizer.join.broadcast-threshold")
+ .defaultValue(1024 * 1024L)
+ .withDescription("Configures the maximum size in bytes for a table that will be broadcast to all worker " +
+ "nodes when performing a join. By setting this value to -1 to disable broadcasting.");
- public static final ConfigOption<Boolean> SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED =
- key("sql.optimizer.distinct-agg.split.enabled")
- .defaultValue(false)
- .withDescription("Tells the optimizer whether to split distinct aggregation " +
- "(e.g. COUNT(DISTINCT col), SUM(DISTINCT col)) into two level. " +
- "The first aggregation is shuffled by an additional key which is calculated using " +
- "the hashcode of distinct_key and number of buckets. This optimization is very useful " +
- "when there is data skew in distinct aggregation and gives the ability to scale-up the job. " +
- "Default is false.");
+ public static final ConfigOption<Boolean> TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED =
+ key("table.optimizer.distinct-agg.split.enabled")
+ .defaultValue(false)
+ .withDescription("Tells the optimizer whether to split distinct aggregation " +
+ "(e.g. COUNT(DISTINCT col), SUM(DISTINCT col)) into two level. " +
+ "The first aggregation is shuffled by an additional key which is calculated using " +
+ "the hashcode of distinct_key and number of buckets. This optimization is very useful " +
+ "when there is data skew in distinct aggregation and gives the ability to scale-up the job. " +
+ "Default is false.");
- public static final ConfigOption<Integer> SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM =
- key("sql.optimizer.distinct-agg.split.bucket-num")
- .defaultValue(1024)
- .withDescription("Configure the number of buckets when splitting distinct aggregation. " +
- "The number is used in the first level aggregation to calculate a bucket key " +
- "'hash_code(distinct_key) % BUCKET_NUM' which is used as an additional group key after splitting.");
+ public static final ConfigOption<Integer> TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM =
+ key("table.optimizer.distinct-agg.split.bucket-num")
+ .defaultValue(1024)
+ .withDescription("Configure the number of buckets when splitting distinct aggregation. " +
+ "The number is used in the first level aggregation to calculate a bucket key " +
+ "'hash_code(distinct_key) % BUCKET_NUM' which is used as an additional group key after splitting.");
- public static final ConfigOption<Boolean> SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED =
- key("sql.optimizer.reuse.sub-plan.enabled")
- .defaultValue(true)
- .withDescription("When it is true, optimizer will try to find out duplicated " +
- "sub-plan and reuse them.");
+ public static final ConfigOption<Boolean> TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED =
+ key("table.optimizer.reuse-sub-plan-enabled")
+ .defaultValue(true)
+ .withDescription("When it is true, the optimizer will try to find out duplicated sub-plans and reuse them.");
- public static final ConfigOption<Boolean> SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED =
- key("sql.optimizer.reuse.table-source.enabled")
- .defaultValue(true)
- .withDescription("When it is true, optimizer will try to find out duplicated table-source and " +
- "reuse them. This works only when " + SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED.key() + " is true.");
+ public static final ConfigOption<Boolean> TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED =
+ key("table.optimizer.reuse-source-enabled")
+ .defaultValue(true)
+ .withDescription("When it is true, the optimizer will try to find out duplicated table sources and " +
+ "reuse them. This works only when " + TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED.key() + " is true.");
- public static final ConfigOption<Boolean> SQL_OPTIMIZER_PREDICATE_PUSHDOWN_ENABLED =
- key("sql.optimizer.predicate-pushdown.enabled")
- .defaultValue(true)
- .withDescription("If it is true, enable predicate pushdown to the FilterableTableSource. " +
- "Default value is true.");
+ public static final ConfigOption<Boolean> TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED =
+ key("table.optimizer.source.predicate-pushdown-enabled")
+ .defaultValue(true)
+ .withDescription("When it is true, the optimizer will push down predicates into the FilterableTableSource. " +
+ "Default value is true.");
- public static final ConfigOption<Boolean> SQL_OPTIMIZER_JOIN_REORDER_ENABLED =
- key("sql.optimizer.join-reorder.enabled")
- .defaultValue(false)
- .withDescription("Enables join reorder in optimizer cbo. Default is disabled.");
+ public static final ConfigOption<Boolean> TABLE_OPTIMIZER_JOIN_REORDER_ENABLED =
+ key("table.optimizer.join-reorder-enabled")
+ .defaultValue(false)
+ .withDescription("Enables join reorder in optimizer. Default is disabled.");
}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java
index 04fcf0d..7bf4367 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java
@@ -96,11 +96,11 @@ public class BatchExecutor extends ExecutorBase {
}
private boolean isShuffleModeAllBatch() {
- String value = tableConfig.getConfiguration().getString(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE);
+ String value = tableConfig.getConfiguration().getString(ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE);
if (value.equalsIgnoreCase(ShuffleMode.BATCH.toString())) {
return true;
} else if (!value.equalsIgnoreCase(ShuffleMode.PIPELINED.toString())) {
- throw new IllegalArgumentException(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE.key() +
+ throw new IllegalArgumentException(ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE.key() +
" can only be set to " + ShuffleMode.BATCH.toString() + " or " + ShuffleMode.PIPELINED.toString());
}
return false;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java
index 96cd692..1d936b6 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java
@@ -39,7 +39,7 @@ public class NodeResourceUtil {
*/
public static int getSourceParallelism(Configuration tableConf, int envParallelism) {
int parallelism = tableConf.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SOURCE_PARALLELISM);
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SOURCE_PARALLELISM);
if (parallelism <= 0) {
parallelism = getOperatorDefaultParallelism(tableConf, envParallelism);
}
@@ -52,7 +52,7 @@ public class NodeResourceUtil {
* @return the config parallelism for sink.
*/
public static int getSinkParallelism(Configuration tableConf) {
- return tableConf.getInteger(ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM);
+ return tableConf.getInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SINK_PARALLELISM);
}
/**
@@ -62,7 +62,7 @@ public class NodeResourceUtil {
*/
public static int getOperatorDefaultParallelism(Configuration tableConf, int envParallelism) {
int parallelism = tableConf.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM);
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM);
if (parallelism <= 0) {
parallelism = envParallelism;
}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/OperatorType.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/OperatorType.java
index a8e066b..be345c9 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/OperatorType.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/OperatorType.java
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions;
/**
* Some dedicated operator type which is used in
- * {@link ExecutionConfigOptions#SQL_EXEC_DISABLED_OPERATORS}.
+ * {@link ExecutionConfigOptions#TABLE_EXEC_DISABLED_OPERATORS}.
*/
public enum OperatorType {
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/AggregatePhaseStrategy.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/AggregatePhaseStrategy.java
index 3b24ec6..c00b500 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/AggregatePhaseStrategy.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/AggregatePhaseStrategy.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.planner.utils;
import org.apache.flink.table.api.config.OptimizerConfigOptions;
/**
- * Aggregate phase strategy which could be specified in {@link OptimizerConfigOptions#SQL_OPTIMIZER_AGG_PHASE_STRATEGY}.
+ * Aggregate phase strategy which could be specified in {@link OptimizerConfigOptions#TABLE_OPTIMIZER_AGG_PHASE_STRATEGY}.
*/
public enum AggregatePhaseStrategy {
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
index d8646e0..d6cc602 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
@@ -29,8 +29,8 @@ import java.util.Set;
import scala.concurrent.duration.Duration;
-import static org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS;
-import static org.apache.flink.table.api.config.OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY;
+import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS;
+import static org.apache.flink.table.api.config.OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY;
/**
* Utility class for {@link TableConfig} related helper functions.
@@ -45,7 +45,10 @@ public class TableConfigUtils {
* @return true if the given operator is disabled.
*/
public static boolean isOperatorDisabled(TableConfig tableConfig, OperatorType operatorType) {
- String value = tableConfig.getConfiguration().getString(SQL_EXEC_DISABLED_OPERATORS);
+ String value = tableConfig.getConfiguration().getString(TABLE_EXEC_DISABLED_OPERATORS);
+ if (value == null) {
+ return false;
+ }
String[] operators = value.split(",");
Set<OperatorType> operatorSets = new HashSet<>();
for (String operator : operators) {
@@ -70,7 +73,7 @@ public class TableConfigUtils {
* @return the aggregate phase strategy
*/
public static AggregatePhaseStrategy getAggPhaseStrategy(TableConfig tableConfig) {
- String aggPhaseConf = tableConfig.getConfiguration().getString(SQL_OPTIMIZER_AGG_PHASE_STRATEGY).trim();
+ String aggPhaseConf = tableConfig.getConfiguration().getString(TABLE_OPTIMIZER_AGG_PHASE_STRATEGY).trim();
if (aggPhaseConf.isEmpty()) {
return AggregatePhaseStrategy.AUTO;
} else {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/NestedLoopJoinCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/NestedLoopJoinCodeGenerator.scala
index 933ab6c..09f6c9b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/NestedLoopJoinCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/NestedLoopJoinCodeGenerator.scala
@@ -18,11 +18,11 @@
package org.apache.flink.table.planner.codegen
+import org.apache.flink.configuration.MemorySize
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.dataformat.{BaseRow, JoinedRow}
-import org.apache.flink.table.planner.codegen.CodeGenUtils.{BASE_ROW, BINARY_ROW, DEFAULT_INPUT1_TERM, DEFAULT_INPUT2_TERM, className, newName}
+import org.apache.flink.table.planner.codegen.CodeGenUtils._
import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.{INPUT_SELECTION, generateCollect}
-import org.apache.flink.table.planner.plan.nodes.resource.NodeResourceUtil
import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
import org.apache.flink.table.runtime.operators.join.FlinkJoinType
import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer
@@ -70,8 +70,8 @@ class NestedLoopJoinCodeGenerator(
val isFirstRow = newName("isFirstRow")
val isBinaryRow = newName("isBinaryRow")
- val externalBufferMemorySize = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM) * NodeResourceUtil.SIZE_IN_MB
+ val externalBufferMemorySize = MemorySize.parse(config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY)).getBytes
if (singleRowJoin) {
ctx.addReusableMember(s"$BASE_ROW $buildRow = null;")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
index aa6705d..8555510 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
@@ -159,7 +159,7 @@ class FlinkRelMdColumnNullCount private extends MetadataHandler[ColumnNullCount]
val rexBuilder = rel.getCluster.getRexBuilder
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
- FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+ FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
val conjunctions = RelOptUtil.conjunctions(cnf)
val notNullPredicatesAtIndexField = conjunctions.exists {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
index eead6a3..17e2d44 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
@@ -205,7 +205,7 @@ class FlinkRelMdDistinctRowCount private extends MetadataHandler[BuiltInMetadata
val rexBuilder = rel.getCluster.getRexBuilder
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
- FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+ FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
val conjunctions = RelOptUtil.conjunctions(cnf)
val conjunctionsWithoutExpandId = conjunctions.filterNot { c =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
index 793844b..fa54944 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
@@ -73,7 +73,7 @@ class FlinkRelMdDistribution private extends MetadataHandler[FlinkDistribution]
def flinkDistribution(sort: Sort, mq: RelMetadataQuery): FlinkRelDistribution = {
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(sort)
val enableRangeSort = tableConfig.getConfiguration.getBoolean(
- BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
+ BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED)
if ((sort.getCollation.getFieldCollations.nonEmpty &&
sort.fetch == null && sort.offset == null) && enableRangeSort) {
//If Sort is global sort, and the table config allows the range partition.
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
index 5c2641a..dd67610 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
@@ -177,7 +177,7 @@ class FlinkRelMdRowCount private extends MetadataHandler[BuiltInMetadata.RowCoun
val config = rel.getCluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
val parallelism = (inputRowCnt /
config.getConfiguration.getLong(
- FlinkRelMdRowCount.SQL_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
+ FlinkRelMdRowCount.TABLE_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
if (parallelism == 1) {
ndvOfGroupKeysOnGlobalAgg
} else if (grouping.isEmpty) {
@@ -447,8 +447,8 @@ object FlinkRelMdRowCount {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_ROWS_PER_LOCALAGG: ConfigOption[JLong] =
- key("sql.optimizer.rows-per-local-agg")
+ val TABLE_OPTIMIZER_ROWS_PER_LOCALAGG: ConfigOption[JLong] =
+ key("table.optimizer.rows-per-local-agg")
.defaultValue(JLong.valueOf(1000000L))
.withDescription("Sets estimated number of records that one local-agg processes. " +
"Optimizer will infer whether to use local/global aggregate according to it.")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
index 75105c4..600ad0a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
@@ -58,7 +58,7 @@ class SelectivityEstimator(rel: RelNode, mq: FlinkRelMetadataQuery)
private val rexBuilder = rel.getCluster.getRexBuilder
private val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
private val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
- FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+ FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
// these default values is referred to RelMdUtil#guessSelectivity
private[flink] val defaultComparisonSelectivity = Some(0.5d)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala
index f02c894..7a2b133 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala
@@ -188,9 +188,9 @@ abstract class CommonLookupJoin(
val operator = if (lookupableTableSource.isAsyncEnabled) {
val asyncBufferCapacity= config.getConfiguration
- .getInteger(ExecutionConfigOptions.SQL_EXEC_LOOKUP_ASYNC_BUFFER_CAPACITY)
+ .getInteger(ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_BUFFER_CAPACITY)
val asyncTimeout = getMillisecondFromConfigDuration(config,
- ExecutionConfigOptions.SQL_EXEC_LOOKUP_ASYNC_TIMEOUT)
+ ExecutionConfigOptions.TABLE_EXEC_ASYNC_LOOKUP_TIMEOUT)
val asyncLookupFunction = lookupableTableSource
.getAsyncLookupFunction(lookupFieldNamesInOrder)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
index c1c01bf..fef0ca2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
@@ -18,10 +18,10 @@
package org.apache.flink.table.planner.plan.nodes.logical
-import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT
+import org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT
import org.apache.flink.table.planner.calcite.FlinkContext
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
-import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED
import org.apache.flink.table.planner.plan.utils.SortUtil
import org.apache.calcite.plan._
@@ -105,8 +105,8 @@ class FlinkLogicalSortBatchConverter extends ConverterRule(
val sort = rel.asInstanceOf[LogicalSort]
val newInput = RelOptRule.convert(sort.getInput, FlinkConventions.LOGICAL)
val config = sort.getCluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
- val enableRangeSort = config.getConfiguration.getBoolean(SQL_EXEC_SORT_RANGE_ENABLED)
- val limitValue = config.getConfiguration.getInteger(SQL_EXEC_SORT_DEFAULT_LIMIT)
+ val enableRangeSort = config.getConfiguration.getBoolean(TABLE_EXEC_SORT_RANGE_ENABLED)
+ val limitValue = config.getConfiguration.getInteger(TABLE_EXEC_SORT_DEFAULT_LIMIT)
val (offset, fetch) = if (sort.fetch == null && sort.offset == null
&& !enableRangeSort && limitValue > 0) {
//force the sort add limit
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExchange.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExchange.scala
index dfacbc5..b82ead7 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExchange.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExchange.scala
@@ -90,7 +90,7 @@ class BatchExecExchange(
requiredShuffleMode match {
case Some(mode) if mode eq ShuffleMode.BATCH => mode
case _ =>
- if (tableConf.getString(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE)
+ if (tableConf.getString(ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE)
.equalsIgnoreCase(ShuffleMode.BATCH.toString)) {
ShuffleMode.BATCH
} else {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregate.scala
index 6b681d7..481f387 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregate.scala
@@ -114,7 +114,7 @@ class BatchExecHashAggregate(
// If partialKey is enabled, try to use partial key to satisfy the required distribution
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
}
case _ => false
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
index 53b318e..cf8d0ee 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
@@ -34,7 +34,6 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil.transformToBatchA
import org.apache.flink.table.planner.plan.utils.FlinkRelMdUtil
import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
-
import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rel.`type`.RelDataType
@@ -42,6 +41,7 @@ import org.apache.calcite.rel.core.AggregateCall
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.tools.RelBuilder
import org.apache.calcite.util.Util
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -136,8 +136,9 @@ abstract class BatchExecHashAggregateBase(
AggWithoutKeysCodeGenerator.genWithoutKeys(
ctx, relBuilder, aggInfos, inputType, outputType, isMerge, isFinal, "NoGrouping")
} else {
- managedMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM)
+ val memText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY)
+ managedMemoryInMB = MemorySize.parse(memText).getMebiBytes
val managedMemory = managedMemoryInMB * NodeResourceUtil.SIZE_IN_MB
new HashAggCodeGenerator(
ctx, relBuilder, aggInfos, inputType, outputType, grouping, auxGrouping, isMerge, isFinal
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashJoin.scala
index 49d333d..94050ff 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashJoin.scala
@@ -36,13 +36,13 @@ import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, JoinUtil}
import org.apache.flink.table.runtime.operators.join.{HashJoinOperator, HashJoinType}
import org.apache.flink.table.runtime.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
import org.apache.flink.table.types.logical.RowType
-
import org.apache.calcite.plan._
import org.apache.calcite.rel.core._
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
import org.apache.calcite.rex.RexNode
import org.apache.calcite.util.Util
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -201,8 +201,9 @@ class BatchExecHashJoin(
val rType = rInput.getOutputType.asInstanceOf[BaseRowTypeInfo].toRowType
val keyType = RowType.of(leftKeys.map(lType.getChildren().get(_)): _*)
- val managedMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM)
+ val memText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY)
+ val managedMemoryInMB = MemorySize.parse(memText).getMebiBytes
val managedMemory = managedMemoryInMB * NodeResourceUtil.SIZE_IN_MB
val condFunc = JoinUtil.generateConditionFunction(
config, cluster.getRexBuilder, getJoinInfo, lType, rType)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
index c24c6f6..0d3dd07 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
@@ -37,13 +37,13 @@ import org.apache.flink.table.planner.plan.utils.FlinkRelMdUtil
import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
import org.apache.flink.table.runtime.operators.aggregate.BytesHashMap
import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
-
import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.AggregateCall
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -131,12 +131,13 @@ abstract class BatchExecHashWindowAggregateBase(
aggCallToAggFunction.map(_._1), aggInputRowType)
val groupBufferLimitSize = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
+ ExecutionConfigOptions.TABLE_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
val (windowSize: Long, slideSize: Long) = WindowCodeGenerator.getWindowDef(window)
- val managedMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM)
+ val memText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY)
+ val managedMemoryInMB = MemorySize.parse(memText).getMebiBytes
val managedMemory = managedMemoryInMB * NodeResourceUtil.SIZE_IN_MB
val generatedOperator = new HashWindowCodeGenerator(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
index 6921553..27e3629 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
@@ -31,12 +31,12 @@ import org.apache.flink.table.planner.plan.nodes.ExpressionFormat
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode
import org.apache.flink.table.planner.plan.nodes.resource.NodeResourceUtil
import org.apache.flink.table.runtime.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
-
import org.apache.calcite.plan._
import org.apache.calcite.rel.core._
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
import org.apache.calcite.rex.RexNode
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -155,8 +155,9 @@ class BatchExecNestedLoopJoin(
val externalBufferMemoryInMb: Int = if (singleRowJoin) {
0
} else {
- planner.getTableConfig.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM)
+ val mem = planner.getTableConfig.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY)
+ MemorySize.parse(mem).getMebiBytes
}
val resourceSpec = NodeResourceUtil.fromManagedMem(externalBufferMemoryInMb)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecOverAggregate.scala
index a4b7cb6..90dde6f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecOverAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecOverAggregate.scala
@@ -47,7 +47,6 @@ import org.apache.flink.table.runtime.operators.over.frame.{InsensitiveOverFrame
import org.apache.flink.table.runtime.operators.over.{BufferDataOverWindowOperator, NonBufferOverWindowOperator}
import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
import org.apache.flink.table.types.logical.LogicalTypeRoot.{BIGINT, INTEGER, SMALLINT}
-
import org.apache.calcite.plan._
import org.apache.calcite.rel.RelDistribution.Type._
import org.apache.calcite.rel._
@@ -60,6 +59,7 @@ import org.apache.calcite.sql.SqlKind
import org.apache.calcite.sql.fun.SqlLeadLagAggFunction
import org.apache.calcite.tools.RelBuilder
import org.apache.calcite.util.ImmutableIntList
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -276,7 +276,7 @@ class BatchExecOverAggregate(
if (isAllFieldsFromInput) {
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
if (tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)) {
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)) {
ImmutableIntList.of(grouping: _*).containsAll(requiredDistribution.getKeys)
} else {
requiredDistribution.getKeys == ImmutableIntList.of(grouping: _*)
@@ -410,8 +410,9 @@ class BatchExecOverAggregate(
new NonBufferOverWindowOperator(aggHandlers, genComparator, resetAccumulators)
} else {
val windowFrames = createOverWindowFrames(config)
- managedMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM)
+ val memText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY)
+ managedMemoryInMB = MemorySize.parse(memText).getMebiBytes
new BufferDataOverWindowOperator(
managedMemoryInMB * NodeResourceUtil.SIZE_IN_MB,
windowFrames,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecRank.scala
index 08fd4a6..7ecfed4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecRank.scala
@@ -144,7 +144,7 @@ class BatchExecRank(
// If partialKey is enabled, try to use partial key to satisfy the required distribution
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
partialKeyEnabled && partitionKeyList.containsAll(shuffleKeys)
}
case _ => false
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSort.scala
index 84113c6..7d0226a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSort.scala
@@ -32,12 +32,12 @@ import org.apache.flink.table.planner.plan.nodes.resource.NodeResourceUtil
import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, RelExplainUtil, SortUtil}
import org.apache.flink.table.runtime.operators.sort.SortOperator
import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
-
import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
import org.apache.calcite.rel.core.Sort
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelCollation, RelNode, RelWriter}
import org.apache.calcite.rex.RexNode
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -116,8 +116,9 @@ class BatchExecSort(
val keyTypes = keys.map(inputType.getTypeAt)
val codeGen = new SortCodeGenerator(conf, keys, keyTypes, orders, nullsIsLast)
- val managedMemoryInMB = conf.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM)
+ val memText = conf.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SORT_MEMORY)
+ val managedMemoryInMB = MemorySize.parse(memText).getMebiBytes
val managedMemory = managedMemoryInMB * NodeResourceUtil.SIZE_IN_MB
val operator = new SortOperator(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregate.scala
index 77266b7..9f9e665 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregate.scala
@@ -115,7 +115,7 @@ class BatchExecSortAggregate(
// If partialKey is enabled, try to use partial key to satisfy the required distribution
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
}
case _ => false
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
index 8c9ffb6..d4f9d22 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
@@ -36,12 +36,12 @@ import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, FlinkRelOptUti
import org.apache.flink.table.runtime.operators.join.{FlinkJoinType, SortMergeJoinOperator}
import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
import org.apache.flink.table.types.logical.RowType
-
import org.apache.calcite.plan._
import org.apache.calcite.rel.core._
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelCollationTraitDef, RelNode, RelWriter}
import org.apache.calcite.rex.RexNode
+import org.apache.flink.configuration.MemorySize
import java.util
@@ -219,12 +219,14 @@ class BatchExecSortMergeJoin(
leftType,
rightType)
- val externalBufferMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM)
+ val externalBufferMemText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY)
+ val externalBufferMemoryInMB = MemorySize.parse(externalBufferMemText).getMebiBytes
val externalBufferMemory = externalBufferMemoryInMB * NodeResourceUtil.SIZE_IN_MB
- val sortMemoryInMB = config.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM)
+ val sortMemText = config.getConfiguration.getString(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SORT_MEMORY)
+ val sortMemoryInMB = MemorySize.parse(sortMemText).getMebiBytes
val sortMemory = sortMemoryInMB * NodeResourceUtil.SIZE_IN_MB
def newSortGen(originalKeys: Array[Int], t: RowType): SortCodeGenerator = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
index 48ec916..52707f1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
@@ -119,7 +119,7 @@ abstract class BatchExecSortWindowAggregateBase(
aggCallToAggFunction.map(_._1), aggInputRowType)
val groupBufferLimitSize = planner.getTableConfig.getConfiguration.getInteger(
- ExecutionConfigOptions.SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
+ ExecutionConfigOptions.TABLE_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
val (windowSize: Long, slideSize: Long) = WindowCodeGenerator.getWindowDef(window)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDeduplicate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDeduplicate.scala
index 3b1e135..251d137 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDeduplicate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDeduplicate.scala
@@ -115,7 +115,7 @@ class StreamExecDeduplicate(
val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
val tableConfig = planner.getTableConfig
val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
val operator = if (isMiniBatchEnabled) {
val exeConfig = planner.getExecEnv.getConfig
val rowSerializer = rowTypeInfo.createSerializer(exeConfig)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
index 008523b..9676ab9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
@@ -164,7 +164,7 @@ class StreamExecGlobalGroupAggregate(
.generateRecordEqualiser("GroupAggValueEqualiser")
val operator = if (tableConfig.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)) {
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)) {
val aggFunction = new MiniBatchGlobalGroupAggFunction(
localAggsHandler,
globalAggsHandler,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
index 4a3f953..6f8b906 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
@@ -168,7 +168,7 @@ class StreamExecGroupAggregate(
val inputCountIndex = aggInfoList.getIndexOfCountStar
val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
val operator = if (isMiniBatchEnabled) {
val aggFunction = new MiniBatchGroupAggFunction(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLimit.scala
index c1365e4..34078c7 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLimit.scala
@@ -127,7 +127,8 @@ class StreamExecLimit(
tableConfig, "AlwaysEqualsComparator", Array(), Array(), Array(), Array())
val processFunction = if (generateRetraction) {
- val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+ val cacheSize = tableConfig.getConfiguration.getLong(
+ StreamExecRank.TABLE_EXEC_TOPN_CACHE_SIZE)
new AppendOnlyTopNFunction(
minIdleStateRetentionTime,
maxIdleStateRetentionTime,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecRank.scala
index 2b0915b..ebf1472 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecRank.scala
@@ -154,7 +154,7 @@ class StreamExecRank(
val sortKeyComparator = ComparatorCodeGenerator.gen(tableConfig, "StreamExecSortComparator",
sortFields.indices.toArray, sortKeyType.getLogicalTypes, sortDirections, nullsIsLast)
val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
- val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+ val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.TABLE_EXEC_TOPN_CACHE_SIZE)
val minIdleStateRetentionTime = tableConfig.getMinIdleStateRetentionTime
val maxIdleStateRetentionTime = tableConfig.getMaxIdleStateRetentionTime
@@ -246,8 +246,8 @@ object StreamExecRank {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_TOPN_CACHE_SIZE: ConfigOption[JLong] =
- key("sql.exec.topn.cache.size")
+ val TABLE_EXEC_TOPN_CACHE_SIZE: ConfigOption[JLong] =
+ key("table.exec.topn.cache-size")
.defaultValue(JLong.valueOf(10000L))
.withDescription("TopN operator has a cache which caches partial state contents to reduce" +
" state access. Cache size is the number of records in each TopN task.")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSort.scala
index 2eee6a8..796491b5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSort.scala
@@ -114,7 +114,7 @@ class StreamExecSort(
override protected def translateToPlanInternal(
planner: StreamPlanner): Transformation[BaseRow] = {
val config = planner.getTableConfig
- if (!config.getConfiguration.getBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED)) {
+ if (!config.getConfiguration.getBoolean(StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED)) {
throw new TableException("Sort on a non-time-attribute field is not supported.")
}
@@ -146,8 +146,8 @@ object StreamExecSort {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_SORT_NON_TEMPORAL_ENABLED: ConfigOption[JBoolean] =
- key("sql.exec.sort.non-temporal.enabled")
+ val TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED: ConfigOption[JBoolean] =
+ key("table.exec.non-temporal-sort.enabled")
.defaultValue(JBoolean.valueOf(false))
.withDescription("Set whether to enable universal sort for stream. When it is false, " +
"universal sort can't use for stream, default false. Just for testing.")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSortLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSortLimit.scala
index 871ab8b..3ef6ac5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSortLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSortLimit.scala
@@ -151,7 +151,7 @@ class StreamExecSortLimit(
sortDirections,
nullsIsLast)
val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
- val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+ val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.TABLE_EXEC_TOPN_CACHE_SIZE)
val minIdleStateRetentionTime = tableConfig.getMinIdleStateRetentionTime
val maxIdleStateRetentionTime = tableConfig.getMaxIdleStateRetentionTime
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
index 2d5ceed..0db10e4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
@@ -84,7 +84,7 @@ class StreamExecWatermarkAssigner(
} else if (miniBatchInterval.mode == MiniBatchMode.ProcTime) {
val tableConfig = cluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
val miniBatchLatency = getMillisecondFromConfigDuration(tableConfig,
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY)
Preconditions.checkArgument(miniBatchLatency > 0,
"MiniBatch latency must be greater that 0 ms.", null)
s"Proctime, ${miniBatchLatency}ms"
@@ -117,7 +117,7 @@ class StreamExecWatermarkAssigner(
val inferredInterval = getTraitSet.getTrait(
MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval
val idleTimeout = getMillisecondFromConfigDuration(config,
- ExecutionConfigOptions.SQL_EXEC_SOURCE_IDLE_TIMEOUT)
+ ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT)
val (operator, opName) = if (inferredInterval.mode == MiniBatchMode.None ||
inferredInterval.interval == 0) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/RelNodeBlock.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/RelNodeBlock.scala
index dbfa9f8..b588117 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/RelNodeBlock.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/RelNodeBlock.scala
@@ -53,7 +53,7 @@ import scala.collection.mutable
* RelNode, the RelNode is the output node of a new block (or named break-point).
* There are several special cases that a RelNode can not be a break-point.
* (1). UnionAll is not a break-point
- * when [[RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED]] is true
+ * when [[RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED]] is true
* (2). [[TableFunctionScan]], [[Snapshot]] or window aggregate ([[Aggregate]] on a [[Project]]
* with window attribute) are not a break-point because their physical RelNodes are a composite
* RelNode, each of them cannot be optimized individually. e.g. FlinkLogicalTableFunctionScan and
@@ -258,7 +258,7 @@ class RelNodeBlockPlanBuilder private(config: TableConfig) {
private val node2Block = new util.IdentityHashMap[RelNode, RelNodeBlock]()
private val isUnionAllAsBreakPointDisabled = config.getConfiguration.getBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED)
/**
* Decompose the [[RelNode]] plan into many [[RelNodeBlock]]s,
@@ -381,15 +381,15 @@ object RelNodeBlockPlanBuilder {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED: ConfigOption[JBoolean] =
- key("sql.optimizer.unionall-as-breakpoint.disabled")
+ val TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED: ConfigOption[JBoolean] =
+ key("table.optimizer.union-all-as-breakpoint-disabled")
.defaultValue(JBoolean.valueOf(false))
.withDescription("Disable union-all node as breakpoint when constructing common sub-graph.")
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED: ConfigOption[JBoolean] =
- key("sql.optimizer.reuse.optimize-block.with-digest.enabled")
+ val TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED: ConfigOption[JBoolean] =
+ key("table.optimizer.reuse-optimize-block-with-digest-enabled")
.defaultValue(JBoolean.valueOf(false))
.withDescription("When true, the optimizer will try to find out duplicated sub-plan by " +
"digest to build optimize block(a.k.a. common sub-graph). " +
@@ -430,7 +430,7 @@ object RelNodeBlockPlanBuilder {
*/
private def reuseRelNodes(relNodes: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
val findOpBlockWithDigest = tableConfig.getConfiguration.getBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED)
if (!findOpBlockWithDigest) {
return relNodes
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
index b14fdc2..5f3ee52 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
@@ -68,9 +68,9 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
}
sinkBlock.setUpdateAsRetraction(retractionFromRoot)
val miniBatchInterval: MiniBatchInterval = if (config.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)) {
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)) {
val miniBatchLatency = getMillisecondFromConfigDuration(config,
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY)
Preconditions.checkArgument(miniBatchLatency > 0,
"MiniBatch Latency must be greater than 0 ms.", null)
MiniBatchInterval(miniBatchLatency, MiniBatchMode.ProcTime)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
index 160c09a..3dfe2ec 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
@@ -137,7 +137,7 @@ object FlinkBatchProgram {
.build())
// join reorder
- if (config.getBoolean(OptimizerConfigOptions.SQL_OPTIMIZER_JOIN_REORDER_ENABLED)) {
+ if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
chainedProgram.addLast(
JOIN_REORDER,
FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext]
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
index 9a734bc..422ec7a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
@@ -133,7 +133,7 @@ object FlinkStreamProgram {
.build())
// join reorder
- if (config.getBoolean(OptimizerConfigOptions.SQL_OPTIMIZER_JOIN_REORDER_ENABLED)) {
+ if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
chainedProgram.addLast(
JOIN_REORDER,
FlinkGroupProgramBuilder.newBuilder[StreamOptimizeContext]
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
index c00f31a..df66554 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
@@ -57,11 +57,11 @@ object SubplanReuser {
*/
def reuseDuplicatedSubplan(rels: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
if (!tableConfig.getConfiguration.getBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) {
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) {
return rels
}
val tableSourceReuseEnabled = tableConfig.getConfiguration.getBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED)
val context = new SubplanReuseContext(tableSourceReuseEnabled, rels: _*)
val reuseShuttle = new SubplanReuseShuttle(context)
rels.map(_.accept(reuseShuttle))
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
index aa00ba6..d2933b4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
@@ -63,7 +63,7 @@ class JoinDeriveNullFilterRule
val mq = FlinkRelMetadataQuery.reuseOrCreate(join.getCluster.getMetadataQuery)
val conf = FlinkRelOptUtil.getTableConfigFromContext(join)
val minNullCount = conf.getConfiguration.getLong(
- JoinDeriveNullFilterRule.SQL_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD)
+ JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD)
def createIsNotNullFilter(input: RelNode, keys: ImmutableIntList): RelNode = {
val relBuilder = call.builder()
@@ -98,8 +98,8 @@ object JoinDeriveNullFilterRule {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD: ConfigOption[JLong] =
- key("sql.optimizer.join.null-filter.threshold")
+ val TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD: ConfigOption[JLong] =
+ key("table.optimizer.join.null-filter-threshold")
.defaultValue(JLong.valueOf(2000000L))
.withDescription("To avoid the impact of null values on the single join node, " +
"We will add a null filter (possibly be pushed down) before the join to filter" +
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
index 558c3d6..bd2ab26 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
@@ -49,7 +49,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
override def matches(call: RelOptRuleCall): Boolean = {
val config = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
if (!config.getConfiguration.getBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_PREDICATE_PUSHDOWN_ENABLED)) {
+ OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED)) {
return false
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
index f6ebc73..b2bd3a5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
@@ -19,7 +19,7 @@
package org.apache.flink.table.planner.plan.rules.logical
import org.apache.flink.table.api.TableException
-import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.OptimizerConfigOptions
import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkLogicalRelFactories, FlinkRelBuilder}
import org.apache.flink.table.planner.functions.sql.{FlinkSqlOperatorTable, SqlFirstLastValueAggFunction}
import org.apache.flink.table.planner.plan.PartialFinalType
@@ -115,14 +115,12 @@ class SplitAggregateRule extends RelOptRule(
val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
val agg: FlinkLogicalAggregate = call.rel(0)
- val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
- val splitSkewDistinctAggEnabled = tableConfig.getConfiguration.getBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED)
+ val splitDistinctAggEnabled = tableConfig.getConfiguration.getBoolean(
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED)
val isAllAggSplittable = doAllAggSupportSplit(agg.getAggCallList)
agg.partialFinalType == PartialFinalType.NONE && agg.containsDistinctCall() &&
- isMiniBatchEnabled && splitSkewDistinctAggEnabled && isAllAggSplittable
+ splitDistinctAggEnabled && isAllAggSplittable
}
override def onMatch(call: RelOptRuleCall): Unit = {
@@ -146,7 +144,7 @@ class SplitAggregateRule extends RelOptRule(
val hashFieldsMap: util.Map[Int, Int] = new util.HashMap()
val buckets = tableConfig.getConfiguration.getInteger(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM)
if (hashFieldIndexes.nonEmpty) {
val projects = new util.ArrayList[RexNode](relBuilder.fields)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala
index 0beda8d..55212e2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala
@@ -43,16 +43,16 @@ import scala.collection.JavaConversions._
* +- input of agg
* }}}
* when all aggregate functions are mergeable
- * and [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
+ * and [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
* {{{
* BatchExecHashAggregate
* +- BatchExecExchange (hash by group keys if group keys is not empty, else singleton)
* +- input of agg
* }}}
* when some aggregate functions are not mergeable
- * or [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
+ * or [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
*
- * Notes: if [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
+ * Notes: if [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
* this rule will try to create two possibilities above, and chooses the best one based on cost.
*/
class BatchExecHashAggRule
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashJoinRule.scala
index 1eb9300..e0fc760 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashJoinRule.scala
@@ -146,7 +146,7 @@ class BatchExecHashJoinRule
// add more possibility to only shuffle by partial joinKeys, now only single one
val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
joinInfo.pairs().foreach { pair =>
transformToEquiv(
@@ -178,7 +178,7 @@ class BatchExecHashJoinRule
return (false, false)
}
val threshold = tableConfig.getConfiguration.getLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
joinType match {
case JoinRelType.LEFT => (rightSize <= threshold, false)
case JoinRelType.RIGHT => (leftSize <= threshold, true)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecJoinRuleBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
index 4315840..636978b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
@@ -60,7 +60,7 @@ trait BatchExecJoinRuleBase {
val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(buildRel)
val mq = buildRel.getCluster.getMetadataQuery
val ratioConf = tableConfig.getConfiguration.getDouble(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO)
val inputRows = mq.getRowCount(buildRel)
val ndvOfGroupKey = mq.getDistinctRowCount(
buildRel, ImmutableBitSet.of(distinctKeys: _*), null)
@@ -85,8 +85,8 @@ object BatchExecJoinRuleBase {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO: ConfigOption[JDouble] =
- key("sql.optimizer.semi-anti-join.build-distinct.ndv-ratio")
+ val TABLE_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO: ConfigOption[JDouble] =
+ key("table.optimizer.semi-anti-join.build-distinct.ndv-ratio")
.defaultValue(JDouble.valueOf(0.8))
.withDescription("In order to reduce the amount of data on semi/anti join's" +
" build side, we will add distinct node before semi/anti join when" +
@@ -96,8 +96,8 @@ object BatchExecJoinRuleBase {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED: ConfigOption[JBoolean] =
- key("sql.optimizer.shuffle.partial-key.enabled")
+ val TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED: ConfigOption[JBoolean] =
+ key("table.optimizer.shuffle-by-partial-key-enabled")
.defaultValue(JBoolean.valueOf(false))
.withDescription("Enables shuffling by partial partition keys. " +
"For example, A join with join condition: L.c1 = R.c1 and L.c2 = R.c2. " +
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala
index 702200c..34c6334 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala
@@ -44,7 +44,7 @@ import scala.collection.JavaConversions._
* +- input of agg
* }}}
* when all aggregate functions are mergeable
- * and [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
+ * and [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
* {{{
* BatchExecSortAggregate
* +- Sort (exists if group keys are not empty)
@@ -52,9 +52,9 @@ import scala.collection.JavaConversions._
* +- input of agg
* }}}
* when some aggregate functions are not mergeable
- * or [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
+ * or [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
*
- * Notes: if [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
+ * Notes: if [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
* this rule will try to create two possibilities above, and chooses the best one based on cost.
*/
class BatchExecSortAggRule
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
index 0e6a635..503dbd9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
@@ -110,7 +110,7 @@ class BatchExecSortMergeJoinRule
val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
val candidates = if (tableConfig.getConfiguration.getBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) {
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) {
// add more possibility to remove redundant sort, and longer optimization time
Array((false, false), (true, false), (false, true), (true, true))
} else {
@@ -128,7 +128,7 @@ class BatchExecSortMergeJoinRule
// add more possibility to only shuffle by partial joinKeys, now only single one
val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
joinInfo.pairs().foreach { pair =>
// sort require full key not partial key,
@@ -148,10 +148,10 @@ object BatchExecSortMergeJoinRule {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED: ConfigOption[JBoolean] =
- key("sql.optimizer.sort-merge-join.remove-sort.enabled")
+ val TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED: ConfigOption[JBoolean] =
+ key("table.optimizer.smj.remove-sort-enabled")
.defaultValue(JBoolean.FALSE)
.withDescription("When true, the optimizer will try to remove redundant sort " +
- "for SortMergeJoin. However that will increase optimization time. " +
+ "for sort merge join. However that will increase optimization time. " +
"Default value is false.")
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortRule.scala
index 4f1de31..5b07325 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortRule.scala
@@ -54,7 +54,7 @@ class BatchExecSortRule extends ConverterRule(
val input = sort.getInput
val config = sort.getCluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
val enableRangeSort = config.getConfiguration.getBoolean(
- BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
+ BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED)
val distribution = if (enableRangeSort) {
FlinkRelDistribution.range(sort.getCollation.getFieldCollations)
} else {
@@ -81,8 +81,8 @@ object BatchExecSortRule {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_SORT_RANGE_ENABLED: ConfigOption[JBoolean] =
- key("sql.exec.sort.range.enabled")
+ val TABLE_EXEC_SORT_RANGE_ENABLED: ConfigOption[JBoolean] =
+ key("table.exec.range-sort.enabled")
.defaultValue(JBoolean.valueOf(false))
.withDescription("Sets whether to enable range sort, use range sort to sort all data in" +
" several partitions. When it is false, sorting in only one partition")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
index 9d6f2be..c503670 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
@@ -53,16 +53,16 @@ import scala.collection.JavaConversions._
* +- input of window agg
* }}}
* when all aggregate functions are mergeable
- * and [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
+ * and [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is TWO_PHASE, or
* {{{
* BatchExecHash(or Sort)WindowAggregate
* +- BatchExecExchange (hash by group keys if group keys is not empty, else singleton)
* +- input of window agg
* }}}
* when some aggregate functions are not mergeable
- * or [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
+ * or [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is ONE_PHASE.
*
- * Notes: if [[OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
+ * Notes: if [[OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY]] is NONE,
* this rule will try to create two possibilities above, and chooses the best one based on cost.
* if all aggregate function buffer are fix length, the rule will choose hash window agg.
*/
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
index be69647..040f519 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
@@ -55,7 +55,7 @@ class IncrementalAggregateRule
// whether incremental aggregate is enabled
val incrementalAggEnabled = tableConfig.getConfiguration.getBoolean(
- IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED)
+ IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED)
partialGlobalAgg.partialFinalType == PartialFinalType.PARTIAL &&
finalLocalAgg.partialFinalType == PartialFinalType.FINAL &&
@@ -185,9 +185,9 @@ object IncrementalAggregateRule {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED: ConfigOption[JBoolean] =
- key("sql.optimizer.incremental-agg.enabled")
- .defaultValue(JBoolean.valueOf(false))
+ val TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED: ConfigOption[JBoolean] =
+ key("table.optimizer.incremental-agg-enabled")
+ .defaultValue(JBoolean.valueOf(true))
.withDescription("When both local aggregation and distinct aggregation splitting " +
"are enabled, a distinct aggregation will be optimized into four aggregations, " +
"i.e., local-agg1, global-agg1, local-agg2 and global-Agg2. We can combine global-agg1" +
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
index 90ae255..cc0b16c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
@@ -37,7 +37,7 @@ import scala.collection.JavaConversions._
* 1. supports operators which supports mini-batch and does not require watermark, e.g.
* group aggregate. In this case, [[StreamExecWatermarkAssigner]] with Protime mode will be
* created if not exist, and the interval value will be set as
- * [[ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY]].
+ * [[ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY]].
* 2. supports operators which requires watermark, e.g. window join, window aggregate.
* In this case, [[StreamExecWatermarkAssigner]] already exists, and its MiniBatchIntervalTrait
* will be updated as the merged intervals from its outputs.
@@ -65,7 +65,7 @@ class MiniBatchIntervalInferRule extends RelOptRule(
val inputs = getInputs(rel)
val config = FlinkRelOptUtil.getTableConfigFromContext(rel)
val miniBatchEnabled = config.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
val updatedTrait = rel match {
case _: StreamExecGroupWindowAggregate =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
index ddf70ff..c822ff2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
@@ -76,7 +76,7 @@ class TwoStageOptimizedAggregateRule extends RelOptRule(
isStateBackendDataViews = true)
val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
val isTwoPhaseEnabled = getAggPhaseStrategy(tableConfig) != AggregatePhaseStrategy.ONE_PHASE
isMiniBatchEnabled && isTwoPhaseEnabled &&
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala
index cfb3cd1..544555a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala
@@ -673,10 +673,11 @@ object AggregateUtil extends Enumeration {
* Creates a MiniBatch trigger depends on the config.
*/
def createMiniBatchTrigger(tableConfig: TableConfig): CountBundleTrigger[BaseRow] = {
- val size = tableConfig.getConfiguration.getLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE)
+ val size = tableConfig.getConfiguration.getLong(
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE)
if (size <= 0 ) {
throw new IllegalArgumentException(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE + " must be > 0.")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE + " must be > 0.")
}
new CountBundleTrigger[BaseRow](size)
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
index e3b866e..832f059 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
@@ -148,7 +148,7 @@ object FlinkRelOptUtil {
/** Get max cnf node limit by context of rel */
def getMaxCnfNodeCount(rel: RelNode): Int = {
val tableConfig = getTableConfigFromContext(rel)
- tableConfig.getConfiguration.getInteger(FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+ tableConfig.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
}
/**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala
index b84613d..68e2c5f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala
@@ -44,8 +44,8 @@ object FlinkRexUtil {
// It is a experimental config, will may be removed later.
@Experimental
- private[flink] val SQL_OPTIMIZER_CNF_NODES_LIMIT: ConfigOption[Integer] =
- key("sql.optimizer.cnf.nodes.limit")
+ private[flink] val TABLE_OPTIMIZER_CNF_NODES_LIMIT: ConfigOption[Integer] =
+ key("table.optimizer.cnf-nodes-limit")
.defaultValue(Integer.valueOf(-1))
.withDescription("When converting to conjunctive normal form (CNF, like '(a AND b) OR" +
" c' will be converted to '(a OR c) AND (b OR c)'), fail if the expression exceeds " +
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
index fbe979a..bc2a923 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
@@ -71,7 +71,7 @@ object RankUtil {
// Converts the condition to conjunctive normal form (CNF)
val cnfNodeCount = config.getConfiguration.getInteger(
- FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+ FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
val cnfCondition = FlinkRexUtil.toCnf(rexBuilder, cnfNodeCount, predicate)
// split the condition into sort limit condition and other condition
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowEmitStrategy.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowEmitStrategy.scala
index 2cb991a..eff8a48 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowEmitStrategy.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowEmitStrategy.scala
@@ -152,13 +152,13 @@ object WindowEmitStrategy {
tableConfig.getMinIdleStateRetentionTime
}
val enableEarlyFireDelay = tableConfig.getConfiguration.getBoolean(
- SQL_EXEC_EMIT_EARLY_FIRE_ENABLED)
+ TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED)
val earlyFireDelay = getMillisecondFromConfigDuration(
- tableConfig, SQL_EXEC_EMIT_EARLY_FIRE_DELAY)
+ tableConfig, TABLE_EXEC_EMIT_EARLY_FIRE_DELAY)
val enableLateFireDelay = tableConfig.getConfiguration.getBoolean(
- SQL_EXEC_EMIT_LATE_FIRE_ENABLED)
+ TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)
val lateFireDelay = getMillisecondFromConfigDuration(
- tableConfig, SQL_EXEC_EMIT_LATE_FIRE_DELAY)
+ tableConfig, TABLE_EXEC_EMIT_LATE_FIRE_DELAY)
new WindowEmitStrategy(
isEventTime,
isSessionWindow,
@@ -171,16 +171,16 @@ object WindowEmitStrategy {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_EMIT_EARLY_FIRE_ENABLED: ConfigOption[Boolean] =
- key("sql.exec.emit.early-fire.enabled")
+ val TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED: ConfigOption[Boolean] =
+ key("table.exec.emit.early-fire.enabled")
.defaultValue(Boolean.valueOf(false))
.withDescription("Specifies whether to enable early-fire emit." +
"Early-fire is an emit strategy before watermark advanced to end of window.")
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_EMIT_EARLY_FIRE_DELAY: ConfigOption[String] =
- key("sql.exec.emit.early-fire.delay")
+ val TABLE_EXEC_EMIT_EARLY_FIRE_DELAY: ConfigOption[String] =
+ key("table.exec.emit.early-fire.delay")
.noDefaultValue
.withDescription("The early firing delay in milli second, early fire is " +
"the emit strategy before watermark advanced to end of window. " +
@@ -190,16 +190,16 @@ object WindowEmitStrategy {
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_EMIT_LATE_FIRE_ENABLED: ConfigOption[Boolean] =
- key("sql.exec.emit.late-fire.enabled")
+ val TABLE_EXEC_EMIT_LATE_FIRE_ENABLED: ConfigOption[Boolean] =
+ key("table.exec.emit.late-fire.enabled")
.defaultValue(Boolean.valueOf(false))
.withDescription("Specifies whether to enable late-fire emit. " +
"Late-fire is an emit strategy after watermark advanced to end of window.")
// It is a experimental config, will may be removed later.
@Experimental
- val SQL_EXEC_EMIT_LATE_FIRE_DELAY: ConfigOption[String] =
- key("sql.exec.emit.late-fire.delay")
+ val TABLE_EXEC_EMIT_LATE_FIRE_DELAY: ConfigOption[String] =
+ key("table.exec.emit.late-fire.delay")
.noDefaultValue
.withDescription("The late firing delay in milli second, late fire is " +
"the emit strategy after watermark advanced to end of window. " +
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala
index cd6033c..6b24f11 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala
@@ -169,7 +169,7 @@ object WindowJoinUtil {
// Converts the condition to conjunctive normal form (CNF)
val cnfCondition = FlinkRexUtil.toCnf(rexBuilder,
- config.getConfiguration.getInteger(FlinkRexUtil.SQL_OPTIMIZER_CNF_NODES_LIMIT),
+ config.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT),
predicate)
// split the condition into time predicates and other predicates
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
index 61a0384..97cba22 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
@@ -51,7 +51,7 @@ public class ShuffleStageParallelismCalculatorTest {
@Before
public void setUp() {
tableConf = new Configuration();
- tableConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 50);
+ tableConf.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 50);
}
@Test
@@ -84,7 +84,7 @@ public class ShuffleStageParallelismCalculatorTest {
@Test
public void testStreamSourceAndCalc() {
- tableConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SOURCE_PARALLELISM, 60);
+ tableConf.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SOURCE_PARALLELISM, 60);
ShuffleStage shuffleStage0 = mock(ShuffleStage.class);
when(shuffleStage0.getMaxParallelism()).thenReturn(60);
StreamExecCalc calc = mock(StreamExecCalc.class);
@@ -105,7 +105,7 @@ public class ShuffleStageParallelismCalculatorTest {
@Test
public void testEnvParallelism() {
- tableConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, -1);
+ tableConf.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, -1);
ShuffleStage shuffleStage0 = mock(ShuffleStage.class);
when(shuffleStage0.getMaxParallelism()).thenReturn(4);
BatchExecCalc calc = mock(BatchExecCalc.class);
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
index d43a1d7..ca4650c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
@@ -64,7 +64,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
def testExplainWithJoin(): Unit = {
// TODO support other join operators when them are supported
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
index 9bc4313..59ae7d6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
@@ -110,9 +110,9 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
util.addTableWithWatermark("T3", util.tableEnv.scan("T1"), "rowtime", 0)
util.addTableWithWatermark("T4", util.tableEnv.scan("T2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
val table = util.tableEnv.sqlQuery(
"""
|SELECT id1, T3.rowtime AS ts, text
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
index b9aad91..5b9665b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
@@ -67,7 +67,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) {
def before(): Unit = {
tableEnv.getConfig
.getConfiguration
- .setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
TestCollectionTableFactory.reset()
TestCollectionTableFactory.isStreaming = isStreamingMode
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
index c722292..a40f3ba 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
@@ -136,7 +136,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
util.tableEnv.registerTable("table1", table1)
val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
@@ -154,7 +154,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10")
@@ -175,7 +175,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10")
@@ -196,7 +196,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks4(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
util.tableEnv.registerTable("table1", table1)
@@ -222,7 +222,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks5(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
// test with non-deterministic udf
util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable")
@@ -242,7 +242,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiLevelViews(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
util.tableEnv.registerTable("TempTable1", table1)
@@ -278,7 +278,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksWithUDTF(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addFunction("split", new TableFunc1)
val sqlQuery1 =
"""
@@ -317,7 +317,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val table = util.tableEnv.sqlQuery(
"SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1")
@@ -337,9 +337,9 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery1 =
@@ -372,7 +372,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1"
@@ -400,7 +400,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion4(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery =
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
index b042c74..2b31551 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
@@ -39,9 +39,9 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_SetExchangeAsBatch(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
val sqlQuery =
"""
|WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -53,11 +53,11 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_AddExchangeAsBatch_HashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a FROM x LIMIT 10)
@@ -69,11 +69,11 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a FROM x LIMIT 10)
@@ -85,11 +85,11 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_SetExchangeAsBatch_SortMergeJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin")
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|
@@ -105,13 +105,13 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_AddExchangeAsBatch_BuildLeftSemiHashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b FROM x LIMIT 10)
@@ -125,7 +125,7 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testSubplanReuse_SetExchangeAsBatch_OverAgg(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
val sqlQuery =
"""
|WITH r1 AS (SELECT SUM(a) OVER (PARTITION BY b ORDER BY b) AS a, b, c FROM x),
@@ -142,11 +142,11 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testReusedNodeIsBarrierNode(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c)
@@ -158,9 +158,9 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testDataStreamReuse_SetExchangeAsBatch(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.a AND t1.b > 10 AND t2.c LIKE 'Test%'"
util.verifyPlan(sqlQuery)
}
@@ -168,7 +168,7 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testDataStreamReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.b"
util.verifyPlan(sqlQuery)
}
@@ -176,7 +176,7 @@ class DeadlockBreakupTest extends TableTestBase {
@Test
def testDataStreamReuse_AddExchangeAsBatch_HashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
val sqlQuery = "SELECT * FROM t INTERSECT SELECT * FROM t"
util.verifyPlan(sqlQuery)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
index a971d98..73c202d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.api.Types
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.plan.stats.TableStats
import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortMergeJoinRule
-import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED
import org.apache.flink.table.planner.plan.stats.FlinkStatistic
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.StringSplit
import org.apache.flink.table.planner.utils.{TableFunc1, TableTestBase}
@@ -60,13 +60,13 @@ class RemoveCollationTest extends TableTestBase {
)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
}
@Test
def testRemoveCollation_OverWindowAgg(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg")
val sqlQuery =
"""
| SELECT
@@ -82,7 +82,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Aggregate(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -94,7 +94,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Aggregate_1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -105,7 +105,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Sort(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT a, b, COUNT(c) AS cnt FROM x GROUP BY a, b)
@@ -117,8 +117,8 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Aggregate_3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x ORDER BY a, b)
@@ -130,7 +130,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Rank_1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
val sqlQuery =
"""
|SELECT a, SUM(b) FROM (
@@ -145,7 +145,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Rank_2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
val sqlQuery =
"""
|SELECT a, b, MAX(c) FROM (
@@ -174,7 +174,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Rank_4(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
val sqlQuery =
"""
|SELECT * FROM (
@@ -189,7 +189,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Rank_Singleton(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
val sqlQuery =
"""
|SELECT COUNT(a), SUM(b) FROM (
@@ -204,7 +204,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_MultipleSortMergeJoins1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sql =
"""
@@ -220,7 +220,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sql =
"""
@@ -236,7 +236,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_MultipleSortMergeJoins2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sql =
"""
@@ -252,7 +252,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sql =
"""
@@ -268,7 +268,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_MultipleSortMergeJoins3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
util.addTableSource("tb1",
Array[TypeInformation[_]](
Types.STRING, Types.STRING, Types.STRING, Types.STRING, Types.STRING),
@@ -344,7 +344,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Correlate1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
@@ -358,7 +358,7 @@ class RemoveCollationTest extends TableTestBase {
@Test
def testRemoveCollation_Correlate2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
@@ -373,7 +373,7 @@ class RemoveCollationTest extends TableTestBase {
def testRemoveCollation_Correlate3(): Unit = {
// do not remove shuffle
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
index 382619e..e8b8519 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
@@ -46,13 +46,13 @@ class RemoveShuffleTest extends TableTestBase {
FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
}
@Test
def testRemoveHashShuffle_OverWindowAgg(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
val sqlQuery =
"""
| SELECT
@@ -69,7 +69,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_MultiOverWindowAgg(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
val sqlQuery =
"""
| SELECT
@@ -86,9 +86,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_OverWindowAgg_PartialKey(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
// push down HashExchange[c] into HashAgg
val sqlQuery =
"""
@@ -106,9 +106,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Agg_PartialKey(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
// push down HashExchange[c] into HashAgg
val sqlQuery =
"""
@@ -121,10 +121,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashAggregate(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -136,10 +136,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashAggregate_1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -151,10 +151,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashAggregate_2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -166,10 +166,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortAggregate(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -181,10 +181,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortAggregate_1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -196,10 +196,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortAggregate_2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -211,9 +211,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortMergeJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -225,9 +225,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortMergeJoin_LOJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -239,9 +239,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortMergeJoin_ROJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -253,7 +253,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_SortMergeJoin_FOJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -265,10 +265,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -280,7 +280,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_BroadcastHashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -292,10 +292,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashJoin_LOJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -307,10 +307,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashJoin_ROJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -322,10 +322,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashJoin_FOJ(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -337,10 +337,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_HashJoin_1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r1 AS (SELECT a, c, sum(b) FROM x group by a, c),
@@ -353,7 +353,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_NestedLoopJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -365,12 +365,12 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Join_PartialKey(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT d, count(f) as cnt FROM y GROUP BY d)
@@ -388,7 +388,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Union(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
val sqlQuery =
"""
|WITH r AS (
@@ -403,7 +403,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
val sqlQuery =
"""
|SELECT * FROM (
@@ -418,9 +418,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank_PartialKey1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
val sqlQuery =
"""
|SELECT a, SUM(b) FROM (
@@ -435,9 +435,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank_PartialKey2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, false)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, false)
val sqlQuery =
"""
|SELECT * FROM (
@@ -452,9 +452,9 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank_PartialKey3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
val sqlQuery =
"""
|SELECT * FROM (
@@ -469,7 +469,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank_Singleton1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
val sqlQuery =
"""
|SELECT * FROM (
@@ -484,7 +484,7 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Rank_Singleton2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
val sqlQuery =
"""
|SELECT * FROM (
@@ -499,10 +499,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Correlate1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
@@ -516,10 +516,10 @@ class RemoveShuffleTest extends TableTestBase {
@Test
def testRemoveHashShuffle_Correlate2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
@@ -534,10 +534,10 @@ class RemoveShuffleTest extends TableTestBase {
def testRemoveHashShuffle_Correlate3(): Unit = {
// do not remove shuffle
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
util.addFunction("split", new TableFunc1)
val sqlQuery =
"""
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
index b07365f..7670822 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
@@ -36,7 +36,7 @@ class SetOperatorsTest extends TableTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
util.addTableSource[(Int, Long, String)]("T1", 'a, 'b, 'c)
util.addTableSource[(Int, Long, String)]("T2", 'd, 'e, 'f)
util.addTableSource[(Int, Long, Int, String, Long)]("T3", 'a, 'b, 'd, 'c, 'e)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
index 9d568ee..e04f9eb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
@@ -54,7 +54,7 @@ class SinkTest extends TableTestBase {
@Test
def testMultiSinks(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
util.tableEnv.registerTable("table1", table1)
val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
index 8771e56..de2692d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.planner.plan.batch.sql
import org.apache.flink.api.scala._
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.api.scala._
-import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED
import org.apache.flink.table.planner.utils.TableTestBase
import org.junit.Test
@@ -31,65 +31,65 @@ class SortLimitTest extends TableTestBase {
private val util = batchTestUtil()
util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
@Test
def testNonRangeSortWithoutOffset(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
}
@Test
def testNonRangeSortWithLimit0(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
}
@Test
def testNonRangeSortOnlyWithOffset(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
}
@Test
def testNoneRangeSortWithOffsetLimit(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
}
@Test
def testNoneRangeSortWithOffsetLimit0(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
}
@Test
def testRangeSortOnWithoutOffset(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
}
@Test
def testRangeSortOnWithLimit0(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
}
@Test
def testRangeSortOnlyWithOffset(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
}
@Test
def testRangeSortWithOffsetLimit(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
}
@Test
def testRangeSortWithOffsetLimit0(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
index 1c3de52..10a19a8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.planner.plan.batch.sql
import org.apache.flink.api.scala._
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.api.scala._
-import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED
import org.apache.flink.table.planner.utils.TableTestBase
import org.junit.Test
@@ -33,43 +33,43 @@ class SortTest extends TableTestBase {
@Test
def testNonRangeSortOnSingleFieldWithoutForceLimit(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
}
@Test
def testNonRangeSortOnMultiFieldsWithoutForceLimit(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- SQL_EXEC_SORT_RANGE_ENABLED, false)
+ TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b")
}
@Test
def testNonRangeSortWithForceLimit(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- SQL_EXEC_SORT_RANGE_ENABLED, false)
+ TABLE_EXEC_SORT_RANGE_ENABLED, false)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
}
@Test
def testRangeSortWithoutForceLimit(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
}
@Test
def testRangeSortWithForceLimit(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
+ ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
index 29a9cad..d20f26e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.api.scala._
import org.apache.flink.table.planner.functions.aggfunctions.FirstValueAggFunction.IntFirstValueAggFunction
import org.apache.flink.table.planner.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction
import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortMergeJoinRule
-import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.planner.plan.rules.physical.batch.BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED
import org.apache.flink.table.planner.plan.utils.OperatorType
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedTableFunctions.{NonDeterministicTableFunc, StringSplit}
@@ -39,9 +39,9 @@ class SubplanReuseTest extends TableTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
}
@@ -49,7 +49,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testDisableSubplanReuse(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
val sqlQuery =
"""
|WITH r AS (
@@ -63,7 +63,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseWithDifferentRowType(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
// can not reuse because of different row-type
val sqlQuery =
"""
@@ -77,7 +77,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testEnableReuseTableSource(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
val sqlQuery =
"""
|WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -89,7 +89,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testDisableReuseTableSource(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
val sqlQuery =
"""
|WITH t AS (SELECT * FROM x, y WHERE x.a = y.d)
@@ -102,9 +102,9 @@ class SubplanReuseTest extends TableTestBase {
def testSubplanReuseOnSourceWithLimit(): Unit = {
// TODO re-check this plan after PushLimitIntoTableSourceScanRule is introduced
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b FROM x LIMIT 10)
@@ -128,7 +128,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnCalc(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b, c FROM x WHERE c LIKE 'test%')
@@ -168,7 +168,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnExchange(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b, c FROM x WHERE c LIKE 'test%')
@@ -182,7 +182,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnHashAggregate(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
val sqlQuery =
"""
|WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c)
@@ -194,7 +194,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnSortAggregate(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
val sqlQuery =
"""
|WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c)
@@ -219,7 +219,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnSort(): Unit = {
- util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+ util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_SORT_RANGE_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c ORDER BY a, b DESC)
@@ -231,7 +231,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnLimit(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b FROM x LIMIT 10)
@@ -274,9 +274,9 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnSortMergeJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+ BatchExecSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -288,7 +288,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnHashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -300,7 +300,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnNestedLoopJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -348,7 +348,7 @@ class SubplanReuseTest extends TableTestBase {
def testSubplanReuseOnCorrelate(): Unit = {
util.addFunction("str_split", new StringSplit())
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b, c, v FROM x, LATERAL TABLE(str_split(c, '-')) AS T(v))
@@ -387,7 +387,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testNestedSubplanReuse(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
val sqlQuery =
"""
|WITH v1 AS (
@@ -423,7 +423,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testBreakupDeadlockOnHashJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a FROM x LIMIT 10)
@@ -435,7 +435,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testBreakupDeadlockOnNestedLoopJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a FROM x LIMIT 10)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
index e25ec23..c972b67 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
@@ -37,9 +37,9 @@ class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe
def before(): Unit = {
// disable sort agg
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
}
override def testMinWithVariableLengthType(): Unit = {
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
index 33f434d..e0dfe72 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
@@ -36,9 +36,9 @@ class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe
def before(): Unit = {
// disable hash agg
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
index 0b021b7..db906eb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
@@ -41,7 +41,7 @@ class WindowAggregateTest(aggStrategy: AggregatePhaseStrategy) extends TableTest
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
util.addFunction("countFun", new CountAggFunction)
util.addTableSource[(Int, Timestamp, Int, Long)]("MyTable", 'a, 'b, 'c, 'd)
util.addTableSource[(Timestamp, Long, Int, String)]("MyTable1", 'ts, 'a, 'b, 'c)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
index 476e4f7..07d9b85 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
@@ -28,9 +28,9 @@ class BroadcastHashJoinTest extends JoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
"SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
index cb61907..6678332 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
@@ -28,9 +28,9 @@ class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
"SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
// the result plan may contains NestedLoopJoin (singleRowJoin)
// which is converted by BatchExecSingleRowJoinRule
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
index bcdbf1f..64e1a8b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
@@ -225,7 +225,7 @@ class LookupJoinTest extends TableTestBase {
@Test
def testReusing(): Unit = {
testUtil.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
val sql1 =
"""
|SELECT b, a, sum(c) c, sum(d) d, PROCTIME() as proctime
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
index 741397d..ff6b724 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
@@ -27,6 +27,6 @@ class NestedLoopJoinTest extends JoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
index 3e8e008..c8548f8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
@@ -27,6 +27,6 @@ class NestedLoopSemiAntiJoinTest extends SemiAntiJoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
index ed01408..c262b51 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
@@ -28,7 +28,7 @@ class ShuffledHashJoinTest extends JoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
"SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
index 6b7b5fa..09d4916 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
@@ -28,7 +28,7 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
"SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
// the result plan may contains NestedLoopJoin (singleRowJoin)
// which is converted by BatchExecSingleRowJoinRule
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
index e917e3d..b8b0a70 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
@@ -28,7 +28,7 @@ class SortMergeJoinTest extends JoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
}
@Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
index 62b8e77..e358c7b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
@@ -28,7 +28,7 @@ class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
// the result plan may contains NestedLoopJoin (singleRowJoin)
// which is converted by BatchExecSingleRowJoinRule
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
index 2751ab2..963334e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
@@ -70,7 +70,7 @@ abstract class JoinReorderTestBase extends TableTestBase {
))).build())
util.getTableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_JOIN_REORDER_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true)
}
@Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/nodes/resource/ExecNodeResourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/nodes/resource/ExecNodeResourceTest.scala
index 616ba4e..2716919 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/nodes/resource/ExecNodeResourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/nodes/resource/ExecNodeResourceTest.scala
@@ -81,7 +81,7 @@ class ExecNodeResourceTest(isBatchMode: Boolean) extends TableTestBase {
@Test
def testConfigSourceParallelism(): Unit = {
testUtil.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SOURCE_PARALLELISM, 100)
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SOURCE_PARALLELISM, 100)
val sqlQuery = "SELECT sum(a) as sum_a, c FROM table3 group by c order by c limit 2"
testUtil.verifyResource(sqlQuery)
}
@@ -116,7 +116,7 @@ class ExecNodeResourceTest(isBatchMode: Boolean) extends TableTestBase {
@Test
def testSinkConfigParallelism(): Unit = {
testUtil.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SINK_PARALLELISM,
25
)
val sqlQuery = "SELECT * FROM table3"
@@ -132,7 +132,7 @@ class ExecNodeResourceTest(isBatchMode: Boolean) extends TableTestBase {
@Test
def testSinkConfigParallelismWhenMax1(): Unit = {
testUtil.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SINK_PARALLELISM,
25
)
val sqlQuery = "SELECT * FROM table3"
@@ -148,7 +148,7 @@ class ExecNodeResourceTest(isBatchMode: Boolean) extends TableTestBase {
@Test
def testSinkConfigParallelismWhenMax2(): Unit = {
testUtil.tableEnv.getConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SINK_PARALLELISM,
25
)
val sqlQuery = "SELECT * FROM table3"
@@ -174,7 +174,7 @@ object ExecNodeResourceTest {
def setResourceConfig(tableConfig: TableConfig): Unit = {
tableConfig.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM,
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM,
18)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
index 2235e30..564d789 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
@@ -53,7 +53,7 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase {
)
util.tableEnv.getConfig.getConfiguration.setLong(
- JoinDeriveNullFilterRule.SQL_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 2000000)
+ JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 2000000)
util.addTableSource("MyTable1",
Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG),
Array("a1", "b1", "c1", "d1", "e1"),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
index d58dbf3..b91649b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
@@ -33,9 +33,8 @@ class SplitAggregateRuleTest extends TableTestBase {
private val util = streamTestUtil()
util.addTableSource[(Long, Int, String)]("MyTable", 'a, 'b, 'c)
util.buildStreamProgram(FlinkStreamProgram.PHYSICAL)
- util.enableMiniBatch()
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
@Test
def testSingleDistinctAgg(): Unit = {
@@ -179,7 +178,7 @@ class SplitAggregateRuleTest extends TableTestBase {
@Test
def testBucketsConfiguration(): Unit = {
util.tableEnv.getConfig.getConfiguration.setInteger(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, 100)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, 100)
val sqlQuery = "SELECT COUNT(DISTINCT c) FROM MyTable"
util.verifyPlan(sqlQuery)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
index b2a9b13..cad9754 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
@@ -40,10 +40,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
@Test
def testRemoveRedundantLocalHashAgg_ShuffleKeyFromJoin(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -55,9 +55,9 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
@Test
def testRemoveRedundantLocalHashAgg_ShuffleKeyFromRank(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
util.tableEnv.getConfig.getConfiguration.setBoolean(
- BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
+ BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
val sqlQuery =
"""
|SELECT a, SUM(b) FROM (
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
index 878dad6..a7f72c9 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
@@ -40,10 +40,10 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase {
@Test
def testRemoveRedundantLocalSortAggWithSort(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
// disable BroadcastHashJoin
util.tableEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -55,7 +55,7 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase {
@Test
def testRemoveRedundantLocalSortAggWithoutSort(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
val sqlQuery =
"""
|WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
index 2f7aa5a..10bc891 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
@@ -36,7 +36,7 @@ class RetractionRulesWithTwoStageAggTest extends TableTestBase {
util.enableMiniBatch()
util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
AggregatePhaseStrategy.TWO_PHASE.toString)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
index d7db92f..bfce5b2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
@@ -121,7 +121,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testSingleSinkSplitOnUnion(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val sqlQuery = "SELECT SUM(a) AS total_sum FROM " +
"(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
@@ -135,7 +135,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
util.tableEnv.registerTable("table1", table1)
val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
@@ -153,7 +153,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10")
@@ -174,7 +174,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10")
@@ -218,7 +218,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinks5(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
// test with non-deterministic udf
util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable")
@@ -238,7 +238,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksWithUDTF(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addFunction("split", new TableFunc1)
val sqlQuery1 =
"""
@@ -277,7 +277,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion1(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val table = util.tableEnv.sqlQuery(
"SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1")
@@ -297,9 +297,9 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion2(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery1 =
@@ -332,7 +332,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion3(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1"
@@ -360,7 +360,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiSinksSplitOnUnion4(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
val sqlQuery =
@@ -454,7 +454,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testMultiLevelViews(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
util.tableEnv.registerTable("TempTable1", table1)
@@ -490,7 +490,7 @@ class DagOptimizationTest extends TableTestBase {
@Test
def testSharedUnionNode(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+ RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
util.tableEnv.registerTable("TempTable1", table1)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
index bc09c8d..ce9e042 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.scala._
import org.apache.flink.table.api.TableConfig
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.api.scala._
-import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{SQL_EXEC_EMIT_EARLY_FIRE_DELAY, SQL_EXEC_EMIT_EARLY_FIRE_ENABLED}
+import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED}
import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase}
import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
@@ -43,13 +43,13 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addDataStream[(Int, String, Long)](
"MyTable2", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
}
@Test
def testMiniBatchOnly(): Unit = {
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable1 GROUP BY b"
util.verifyPlan(sql)
}
@@ -57,7 +57,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
@Test
def testRedundantWatermarkDefinition(): Unit = {
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable3 GROUP BY b"
util.verifyPlan(sql)
@@ -67,7 +67,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
def testWindowWithEarlyFire(): Unit = {
val tableConfig = util.tableEnv.getConfig
tableConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
withEarlyFireDelay(tableConfig, Time.milliseconds(500))
util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
val sql =
@@ -89,7 +89,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
@Test
def testWindowCascade(): Unit = {
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
val sql =
"""
@@ -112,7 +112,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql =
"""
@@ -134,7 +134,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
def testRowtimeRowsOverWithMiniBatch(): Unit = {
util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql =
"""
@@ -157,7 +157,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addTableWithWatermark("RatesHistory", util.tableEnv.scan("MyTable2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
util.addFunction(
"Rates",
@@ -184,7 +184,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql =
"""
@@ -210,7 +210,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
val sql =
"""
@@ -246,7 +246,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
def testMultiOperatorNeedsWatermark3(): Unit = {
util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
val sql =
"""
@@ -280,9 +280,9 @@ class MiniBatchIntervalInferTest extends TableTestBase {
util.addTableWithWatermark("T4", util.tableEnv.scan("T2"), "rowtime", 0)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "500 ms")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "500 ms")
util.tableEnv.getConfig.getConfiguration.setLong(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 300L)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 300L)
val table1 = util.tableEnv.sqlQuery(
"""
@@ -340,13 +340,14 @@ class MiniBatchIntervalInferTest extends TableTestBase {
private def withEarlyFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
val intervalInMillis = interval.toMilliseconds
val preEarlyFireInterval = TableConfigUtils.getMillisecondFromConfigDuration(
- tableConfig, SQL_EXEC_EMIT_EARLY_FIRE_DELAY)
+ tableConfig, TABLE_EXEC_EMIT_EARLY_FIRE_DELAY)
if (preEarlyFireInterval != null && (preEarlyFireInterval != intervalInMillis)) { //
// earlyFireInterval of the two query config is not equal and not the default
throw new RuntimeException("Currently not support different earlyFireInterval configs in " +
"one job")
}
- tableConfig.getConfiguration.setBoolean(SQL_EXEC_EMIT_EARLY_FIRE_ENABLED, true)
- tableConfig.getConfiguration.setString(SQL_EXEC_EMIT_EARLY_FIRE_DELAY, intervalInMillis + " ms")
+ tableConfig.getConfiguration.setBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, true)
+ tableConfig.getConfiguration.setString(
+ TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, intervalInMillis + " ms")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
index 1f6e16c..fcca136 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
@@ -66,9 +66,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
@Test
def testMaxWithRetractOptimizeWithLocalGlobal(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
val query = "SELECT a1, max(a3) from (SELECT a1, a2, max(a3) as a3 FROM A GROUP BY a1, a2) " +
"group by a1"
util.verifyPlanWithTrait(query)
@@ -77,9 +77,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
@Test
def testMinWithRetractOptimizeWithLocalGlobal(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
val query = "SELECT min(a3) from (SELECT a1, a2, min(a3) as a3 FROM A GROUP BY a1, a2)"
util.verifyPlanWithTrait(query)
}
@@ -87,9 +87,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
@Test
def testMinCanNotOptimizeWithLocalGlobal(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
val query =
"SELECT a1, MIN(a3) FROM (SELECT a1, a2, MAX(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1"
util.verifyPlanWithTrait(query)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
index 2dda413..225bc8c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
@@ -36,9 +36,9 @@ class SubplanReuseTest extends TableTestBase {
@Before
def before(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
}
@@ -46,7 +46,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testDisableSubplanReuse(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
val sqlQuery =
"""
|WITH r AS (
@@ -60,7 +60,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseWithDifferentRowType(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
// can not reuse because of different row-type
val sqlQuery =
"""
@@ -74,7 +74,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testEnableReuseTableSource(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
val sqlQuery =
"""
|WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -86,7 +86,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testDisableReuseTableSource(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
val sqlQuery =
"""
|WITH t AS (SELECT * FROM x, y WHERE x.a = y.d)
@@ -182,7 +182,7 @@ class SubplanReuseTest extends TableTestBase {
@Test
def testSubplanReuseOnLimit(): Unit = {
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
val sqlQuery =
"""
|WITH r AS (SELECT a, b FROM x LIMIT 10)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
index 59a48bd..f923a2c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
@@ -64,18 +64,18 @@ class AggregateTest extends TableTestBase {
@Test
def testAggWithMiniBatch(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
util.verifyPlan("SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable GROUP BY b")
}
@Test
def testAggAfterUnionWithMiniBatch(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val query =
"""
|SELECT a, sum(b), count(distinct c)
@@ -97,9 +97,9 @@ class AggregateTest extends TableTestBase {
def testLocalGlobalAggAfterUnion(): Unit = {
// enable local global optimize
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql =
"""
@@ -130,9 +130,9 @@ class AggregateTest extends TableTestBase {
@Test
def testAggWithFilterClauseWithLocalGlobal(): Unit = {
util.tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
val sql =
"""
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
index be58755..7a30103 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
@@ -45,12 +45,12 @@ class DistinctAggregateTest(
util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
util.enableMiniBatch()
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString)
util.tableEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, splitDistinctAggEnabled)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, splitDistinctAggEnabled)
// disable incremental agg
util.tableEnv.getConfig.getConfiguration.setBoolean(
- IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED, false)
+ IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, false)
}
@Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
index 06b06b2..75ebee1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
@@ -38,7 +38,7 @@ class IncrementalAggregateTest(
super.before()
// enable incremental agg
util.tableEnv.getConfig.getConfiguration.setBoolean(
- IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED, true)
+ IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, true)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
index 1a347a5..175d2c0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
@@ -36,7 +36,7 @@ class TwoStageAggregateTest extends TableTestBase {
util.enableMiniBatch()
util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
AggregatePhaseStrategy.TWO_PHASE.toString)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala
index d3cbc12..cc26e52 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala
@@ -36,13 +36,13 @@ class TwoStageAggregateTest extends TableTestBase {
util.tableEnv.getConfig
.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
util.tableEnv.getConfig.getConfiguration
- .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
util.tableEnv.getConfig.getConfiguration
- .setBoolean(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ .setBoolean(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
util.tableEnv.getConfig.getConfiguration
- .setLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3)
+ .setLong(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 3)
util.tableEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
AggregatePhaseStrategy.TWO_PHASE.toString)
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
index ac461dd..ca98714 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
@@ -1236,8 +1236,10 @@ class CalcITCase extends BatchTestBase {
new RowTypeInfo(INT_TYPE_INFO, LONG_TYPE_INFO, BYTE_PRIMITIVE_ARRAY_TYPE_INFO),
"a, b, c",
nullablesOfNullData3)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
- conf.getConfiguration.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+ conf.getConfiguration.setInteger(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
+ conf.getConfiguration.setBoolean(
+ BatchExecSortRule.TABLE_EXEC_SORT_RANGE_ENABLED, true)
checkResult(
"select * from BinaryT order by c",
nullData3.sortBy((x : Row) =>
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/DecimalITCase.scala
index 2358bc2..237d9ea 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/DecimalITCase.scala
@@ -774,7 +774,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin1(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -787,7 +787,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin2(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -800,7 +800,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin3(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -814,7 +814,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin4(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -827,7 +827,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin5(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -840,7 +840,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin6(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -853,7 +853,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin7(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery1(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
s1r(d"1", d"1", 1, 1.0),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala
index eb78d29..d1e929a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.planner.runtime.batch.sql
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.api.scala._
-import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM
+import org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM
import org.apache.flink.table.planner.runtime.batch.sql.join.JoinITCaseHelper
import org.apache.flink.table.planner.runtime.batch.sql.join.JoinType.SortMergeJoin
import org.apache.flink.table.planner.runtime.utils.BatchTestBase
@@ -417,7 +417,7 @@ class MiscITCase extends BatchTestBase {
@Test
def testOrderByAgg(): Unit = {
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ tEnv.getConfig.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
env.setParallelism(1)
checkQuery(
Seq((1, 10), (1, 20), (10, 1), (10, 2)),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
index a81fa23..60dc55b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
@@ -65,7 +65,7 @@ class PartitionableSinkITCase extends BatchTestBase {
env.setParallelism(3)
tEnv.getConfig
.getConfiguration
- .setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
+ .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 3)
registerCollection("nonSortTable", testData, type3, "a, b, c", dataNullables)
registerCollection("sortTable", testData1, type3, "a, b, c", dataNullables)
PartitionableSinkITCase.init()
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SortLimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SortLimitITCase.scala
index 3c042d7..ff8e3ad 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SortLimitITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SortLimitITCase.scala
@@ -109,7 +109,8 @@ class SortLimitITCase extends BatchTestBase {
@Test
def testOrderBehindField(): Unit = {
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ conf.getConfiguration.setInteger(
+ ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
val expected = data3.sortBy((x : Row) => x.getField(2).asInstanceOf[String])
checkResult(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnionITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnionITCase.scala
index 0709cb0..bb79e4c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnionITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnionITCase.scala
@@ -49,7 +49,7 @@ class UnionITCase extends BatchTestBase {
registerCollection("Table5", data5, type5, "d, e, f, g, h", nullablesOfData5)
registerCollection("Table6", data6, type6, "a, b, c", Array(false, false, false))
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
}
@Test
@@ -114,9 +114,9 @@ class UnionITCase extends BatchTestBase {
@Test
def testJoinAfterDifferentTypeUnionAll(): Unit = {
tEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkResult(
"SELECT a, c, g FROM (SELECT t1.a, t1.b, t1.c FROM Table3 t1 UNION ALL" +
"(SELECT a, b, c FROM Table3 ORDER BY a, b, c)), Table5 WHERE b = e",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
index 17d950a..a31250a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
@@ -19,7 +19,7 @@
package org.apache.flink.table.planner.runtime.batch.sql.agg
import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS
+import org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS
import org.apache.flink.table.api.{TableException, Types}
import org.apache.flink.table.planner.calcite.CalciteConfig
import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
@@ -79,7 +79,7 @@ class AggregateJoinTransposeITCase extends BatchTestBase {
tEnv.getConfig.setPlannerConfig(calciteConfig)
// HashJoin is disabled due to translateToPlanInternal method is not implemented yet
- tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_DISABLED_OPERATORS, "HashJoin")
registerCollection("T3", data3, type3, "a, b, c", nullablesOfData3)
registerCollection("MyTable",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
index b930269..f4f1ef0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.planner.runtime.batch.sql.agg
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.table.api.Types
-import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM, SQL_RESOURCE_HASH_AGG_TABLE_MEM}
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{TABLE_EXEC_DISABLED_OPERATORS, TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY}
import org.apache.flink.table.api.config.OptimizerConfigOptions
import org.apache.flink.table.planner.plan.stats.FlinkStatistic
import org.apache.flink.table.planner.runtime.utils.BatchTestBase
@@ -113,12 +113,12 @@ class AggregateReduceGroupingITCase extends BatchTestBase {
FlinkStatistic.builder().uniqueKeys(Set(Set("a6").asJava).asJava).build()
)
// HashJoin is disabled due to translateToPlanInternal method is not implemented yet
- tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_DISABLED_OPERATORS, "HashJoin")
}
@Test
def testSingleAggOnTable_SortAgg(): Unit = {
- tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
testSingleAggOnTable()
checkResult("SELECT a6, b6, max(c6), count(d6), sum(e6) FROM T6 GROUP BY a6, b6",
(0 until 50000).map(i => row(i, 1L, if (i % 500 == 0) null else s"Hello$i", 1L, 10))
@@ -127,21 +127,21 @@ class AggregateReduceGroupingITCase extends BatchTestBase {
@Test
def testSingleAggOnTable_HashAgg_WithLocalAgg(): Unit = {
- tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
tEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE")
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE")
// set smaller parallelism to avoid MemoryAllocationException
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 2)
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
+ tEnv.getConfig.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2)
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY, "2mb")
testSingleAggOnTable()
}
@Test
def testSingleAggOnTable_HashAgg_WithoutLocalAgg(): Unit = {
- tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
tEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE")
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE")
+ tEnv.getConfig.getConfiguration.setString(TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY, "2mb")
testSingleAggOnTable()
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashAggITCase.scala
index d4b2fac..3cc1358 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashAggITCase.scala
@@ -28,6 +28,6 @@ class HashAggITCase
override def prepareAggOp(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
index c3a646d..34c592a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
@@ -28,6 +28,6 @@ class HashDistinctAggregateITCase extends DistinctAggregateITCaseBase {
override def prepareAggOp(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortAggITCase.scala
index 4a66789..f64b0b6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortAggITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, PrimitiveArrayT
import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
import org.apache.flink.api.scala._
import org.apache.flink.table.api.Types
-import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM}
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{TABLE_EXEC_DISABLED_OPERATORS, TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM}
import org.apache.flink.table.functions.AggregateFunction
import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset
import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
@@ -45,7 +45,7 @@ class SortAggITCase
extends AggregateITCaseBase("SortAggregate") {
override def prepareAggOp(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+ TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
registerFunction("countFun", new CountAggFunction())
registerFunction("intSumFun", new IntSumAggFunction())
@@ -63,7 +63,7 @@ class SortAggITCase
@Test
def testBigDataSimpleArrayUDAF(): Unit = {
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ tEnv.getConfig.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
registerFunction("simplePrimitiveArrayUdaf", new SimplePrimitiveArrayUdaf())
registerRange("RangeT", 1000000)
env.setParallelism(1)
@@ -267,7 +267,7 @@ class SortAggITCase
@Test
def testArrayUdaf(): Unit = {
- tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ tEnv.getConfig.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
env.setParallelism(1)
checkResult(
"SELECT myPrimitiveArrayUdaf(a, b) FROM Table3",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
index 25e8fec..b5c51e1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
@@ -36,7 +36,7 @@ class SortDistinctAggregateITCase extends DistinctAggregateITCaseBase {
override def prepareAggOp(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
registerFunction("countFun", new CountAggFunction())
registerFunction("intSumFun", new IntSumAggFunction())
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala
index 53c7266..d2e6220 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.planner.runtime.batch.sql.join
import org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO
import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions._
import org.apache.flink.table.planner.runtime.batch.sql.join.JoinITCaseHelper.disableOtherJoinOpForJoin
import org.apache.flink.table.planner.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
import org.apache.flink.table.planner.runtime.utils.BatchTestBase
@@ -154,9 +154,9 @@ class InnerJoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
@Test
def testBigForSpill(): Unit = {
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_SORT_MEMORY, "1mb")
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY, "2mb")
+ conf.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
val bigData = Random.shuffle(
bigIntStringData.union(bigIntStringData).union(bigIntStringData).union(bigIntStringData))
@@ -171,10 +171,10 @@ class InnerJoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
@Test
def testSortMergeJoinOutputOrder(): Unit = {
if (expectedJoinType == SortMergeJoin) {
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+ conf.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
env.getConfig.setParallelism(1)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_SORT_MEMORY, "1mb")
val bigData = Random.shuffle(
bigIntStringData.union(bigIntStringData).union(bigIntStringData).union(bigIntStringData))
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
index e107eab..cde5f86 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
@@ -756,7 +756,8 @@ class JoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
@Ignore
@Test
def testJoinCollation(): Unit = {
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+ conf.getConfiguration.setString(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_SORT_MEMORY, "1mb")
+
checkResult(
"""
|WITH v1 AS (
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCaseHelper.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCaseHelper.scala
index a17efc7..9baf6ab 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCaseHelper.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCaseHelper.scala
@@ -29,7 +29,7 @@ object JoinITCaseHelper {
def disableBroadcastHashJoin(tEnv: TableEnvironment): Unit = {
tEnv.getConfig.getConfiguration.setLong(
- OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
}
def disableOtherJoinOpForJoin(tEnv: TableEnvironment, expected: JoinType): Unit = {
@@ -42,7 +42,7 @@ object JoinITCaseHelper {
case NestedLoopJoin => "HashJoin, SortMergeJoin"
}
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, disabledOperators)
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, disabledOperators)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/DecimalITCase.scala
index bc65227..1f080a1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/DecimalITCase.scala
@@ -630,7 +630,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin1(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -643,7 +643,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin2(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -656,7 +656,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin3(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -670,7 +670,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin4(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -683,7 +683,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin5(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -696,7 +696,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin6(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
@@ -709,7 +709,7 @@ class DecimalITCase extends BatchTestBase {
@Test
def testJoin7(): Unit = {
tEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+ ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
checkQuery(
Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
s1r(d"1", d"1", 1, 1.0),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortITCase.scala
index 5961757..5d50ec2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortITCase.scala
@@ -67,7 +67,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
@@ -96,7 +96,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
@@ -125,7 +125,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
@@ -154,7 +154,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
@@ -189,7 +189,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
@@ -224,7 +224,7 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
val sink = new TestingRetractSink
tEnv.getConfig.getConfiguration.setBoolean(
- StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+ StreamExecSort.TABLE_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
results.addSink(sink).setParallelism(1)
env.execute()
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala
index d9bd596..66bbfcc 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala
@@ -51,10 +51,10 @@ class SplitAggregateITCase(
if (partialAggMode.isPartialAggEnabled) {
tEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
} else {
tEnv.getConfig.getConfiguration.setBoolean(
- OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, false)
+ OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, false)
}
val data = List(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
index da9df86..132755f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
@@ -25,7 +25,7 @@ import org.apache.flink.api.scala._
import org.apache.flink.table.api.scala._
import org.apache.flink.table.api.{TableConfig, Types}
import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{ConcatDistinctAggFunction, WeightedAvg}
-import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{SQL_EXEC_EMIT_LATE_FIRE_DELAY, SQL_EXEC_EMIT_LATE_FIRE_ENABLED}
+import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_LATE_FIRE_DELAY, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED}
import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode
import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset
import org.apache.flink.table.planner.runtime.utils._
@@ -254,13 +254,14 @@ class WindowAggregateITCase(mode: StateBackendMode)
private def withLateFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
val intervalInMillis = interval.toMilliseconds
val preLateFireInterval = getMillisecondFromConfigDuration(tableConfig,
- SQL_EXEC_EMIT_LATE_FIRE_DELAY)
+ TABLE_EXEC_EMIT_LATE_FIRE_DELAY)
if (preLateFireInterval != null && (preLateFireInterval != intervalInMillis)) {
// lateFireInterval of the two query config is not equal and not the default
throw new RuntimeException(
"Currently not support different lateFireInterval configs in one job")
}
- tableConfig.getConfiguration.setBoolean(SQL_EXEC_EMIT_LATE_FIRE_ENABLED, true)
- tableConfig.getConfiguration.setString(SQL_EXEC_EMIT_LATE_FIRE_DELAY, intervalInMillis + " ms")
+ tableConfig.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true)
+ tableConfig.getConfiguration.setString(
+ TABLE_EXEC_EMIT_LATE_FIRE_DELAY, intervalInMillis + " ms")
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
index 59743ac..844eada 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
import org.apache.flink.streaming.api.transformations.ShuffleMode
-import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions._
import org.apache.flink.table.api.internal.{TableEnvironmentImpl, TableImpl}
import org.apache.flink.table.api.{EnvironmentSettings, SqlParserException, Table, TableConfig, TableEnvironment}
import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, BinaryRowWriter}
@@ -77,14 +77,12 @@ class BatchTestBase extends BatchAbstractTestBase {
@Before
def before(): Unit = {
- conf.getConfiguration.setInteger(
- ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, DEFAULT_PARALLELISM)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
- conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM, 1)
- conf.getConfiguration.setString(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE,
- ShuffleMode.PIPELINED.toString)
+ conf.getConfiguration.setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, DEFAULT_PARALLELISM)
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_HASH_AGG_MEMORY, "2mb")
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_HASH_JOIN_MEMORY, "2mb")
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_SORT_MEMORY, "1mb")
+ conf.getConfiguration.setString(TABLE_EXEC_RESOURCE_EXTERNAL_BUFFER_MEMORY, "1mb")
+ conf.getConfiguration.setString(TABLE_EXEC_SHUFFLE_MODE, ShuffleMode.PIPELINED.toString)
}
/**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithAggTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithAggTestBase.scala
index 29b95e2..02b42ac 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithAggTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithAggTestBase.scala
@@ -43,11 +43,11 @@ class StreamingWithAggTestBase(
tEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
if (aggMode.isLocalAggEnabled) {
tEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
AggregatePhaseStrategy.TWO_PHASE.toString)
} else {
tEnv.getConfig.getConfiguration.setString(
- OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+ OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
AggregatePhaseStrategy.ONE_PHASE.toString)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
index 9991b14..9146ee8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
@@ -17,7 +17,7 @@
*/
package org.apache.flink.table.planner.runtime.utils
-import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_MINIBATCH_ALLOW_LATENCY, SQL_EXEC_MINIBATCH_ENABLED, SQL_EXEC_MINIBATCH_SIZE}
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, TABLE_EXEC_MINIBATCH_ENABLED, TABLE_EXEC_MINIBATCH_SIZE}
import org.apache.flink.table.planner.runtime.utils.StreamingWithMiniBatchTestBase.{MiniBatchMode, MiniBatchOff, MiniBatchOn}
import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
@@ -38,11 +38,11 @@ abstract class StreamingWithMiniBatchTestBase(
val tableConfig = tEnv.getConfig
miniBatch match {
case MiniBatchOn =>
- tableConfig.getConfiguration.setBoolean(SQL_EXEC_MINIBATCH_ENABLED, true)
- tableConfig.getConfiguration.setString(SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
- tableConfig.getConfiguration.setLong(SQL_EXEC_MINIBATCH_SIZE, 3L)
+ tableConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true)
+ tableConfig.getConfiguration.setString(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ tableConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L)
case MiniBatchOff =>
- tableConfig.getConfiguration.removeConfig(SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
+ tableConfig.getConfiguration.removeConfig(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY)
}
}
}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
index 7fb59e1..b117a11 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
@@ -485,7 +485,7 @@ abstract class TableTestUtil(
TestingTableEnvironment.create(setting, catalogManager, tableConfig)
val tableEnv: TableEnvironment = testingTableEnv
tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE, ShuffleMode.PIPELINED.toString)
+ ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE, ShuffleMode.PIPELINED.toString)
private val env: StreamExecutionEnvironment = getPlanner.getExecEnv
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
@@ -741,10 +741,11 @@ case class StreamTableTestUtil(
def enableMiniBatch(): Unit = {
tableEnv.getConfig.getConfiguration.setBoolean(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
tableEnv.getConfig.getConfiguration.setString(
- ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
- tableEnv.getConfig.getConfiguration.setLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3L)
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+ tableEnv.getConfig.getConfiguration.setLong(
+ ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 3L)
}
def createAppendTableSink(
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
index 2e71061..7fbae8a 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.hashtable;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
@@ -180,12 +181,12 @@ public abstract class BaseHybridHashTable implements MemorySegmentPool {
boolean tryDistinctBuildRow) {
//TODO: read compression config from configuration
- this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED);
+ this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED);
this.compressionCodecFactory = this.compressionEnable
- ? BlockCompressionFactory.createBlockCompressionFactory(
- conf.getString(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_CODEC))
+ ? BlockCompressionFactory.createBlockCompressionFactory(BlockCompressionFactory.CompressionFactoryName.LZ4.toString())
: null;
- this.compressionBlockSize = conf.getInteger(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_BLOCK_SIZE);
+ this.compressionBlockSize = (int) MemorySize.parse(
+ conf.getString(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE)).getBytes();
this.owner = owner;
this.avgRecordLen = avgRecordLen;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
index ef81ddb..da99301 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
@@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.operators.sort;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.AlgorithmOptions;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
@@ -211,14 +212,15 @@ public class BinaryExternalSorter implements Sorter<BinaryRow> {
NormalizedKeyComputer normalizedKeyComputer,
RecordComparator comparator, Configuration conf,
float startSpillingFraction) throws IOException {
- int maxNumFileHandles = conf.getInteger(ExecutionConfigOptions.SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM);
- this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED);
+ int maxNumFileHandles = conf.getInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES);
+ this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED);
this.compressionCodecFactory = this.compressionEnable
? BlockCompressionFactory.createBlockCompressionFactory(
- conf.getString(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_CODEC))
+ BlockCompressionFactory.CompressionFactoryName.LZ4.toString())
: null;
- compressionBlockSize = conf.getInteger(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_BLOCK_SIZE);
- asyncMergeEnable = conf.getBoolean(ExecutionConfigOptions.SQL_EXEC_SORT_ASYNC_MERGE_ENABLED);
+ this.compressionBlockSize = (int) MemorySize.parse(
+ conf.getString(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE)).getBytes();
+ asyncMergeEnable = conf.getBoolean(ExecutionConfigOptions.TABLE_EXEC_SORT_ASYNC_MERGE_ENABLED);
checkArgument(maxNumFileHandles >= 2);
checkNotNull(ioManager);
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
index ca969ca..55b0587 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
@@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.operators.sort;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
@@ -98,13 +99,14 @@ public class BufferedKVExternalSorter {
this.comparator = comparator;
this.pageSize = pageSize;
this.sorter = new QuickSort();
- this.maxNumFileHandles = conf.getInteger(ExecutionConfigOptions.SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM);
- this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED);
+ this.maxNumFileHandles = conf.getInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES);
+ this.compressionEnable = conf.getBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED);
this.compressionCodecFactory = this.compressionEnable
? BlockCompressionFactory.createBlockCompressionFactory(
- conf.getString(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_CODEC))
+ BlockCompressionFactory.CompressionFactoryName.LZ4.toString())
: null;
- this.compressionBlockSize = conf.getInteger(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_BLOCK_SIZE);
+ this.compressionBlockSize = (int) MemorySize.parse(
+ conf.getString(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_BLOCK_SIZE)).getBytes();
this.ioManager = ioManager;
this.enumerator = this.ioManager.createChannelEnumerator();
this.channelManager = new SpillChannelManager();
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
index ddc03f6..5cfa840 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
@@ -89,7 +89,7 @@ public class BinaryHashTableTest {
this.ioManager = new IOManagerAsync();
conf = new Configuration();
- conf.setBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED, useCompress);
+ conf.setBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED, useCompress);
}
@After
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
index ea03cea..9edc5d6 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
@@ -82,7 +82,7 @@ public class LongHashTableTest {
this.ioManager = new IOManagerAsync();
conf = new Configuration();
- conf.setBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED, useCompress);
+ conf.setBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED, useCompress);
}
private class MyHashTable extends LongHybridHashTable {
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java
index b6d7081..23dc50da 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java
@@ -67,10 +67,10 @@ public class BinaryExternalSorterTest {
ioManager = new IOManagerAsync();
conf = new Configuration();
if (!spillCompress) {
- conf.setBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED, false);
+ conf.setBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED, false);
}
if (asyncMerge) {
- conf.setBoolean(ExecutionConfigOptions.SQL_EXEC_SORT_ASYNC_MERGE_ENABLED, true);
+ conf.setBoolean(ExecutionConfigOptions.TABLE_EXEC_SORT_ASYNC_MERGE_ENABLED, true);
}
}
@@ -96,7 +96,7 @@ public class BinaryExternalSorterTest {
public void beforeTest() {
this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.serializer = new BinaryRowSerializer(2);
- this.conf.setInteger(ExecutionConfigOptions.SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM, 128);
+ this.conf.setInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES, 128);
}
@After
@@ -310,7 +310,7 @@ public class BinaryExternalSorterTest {
LOG.debug("initializing sortmerger");
long minMemorySize = memoryManager.computeNumberOfPages(0.01) * MemoryManager.DEFAULT_PAGE_SIZE;
- conf.setInteger(ExecutionConfigOptions.SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM, 8);
+ conf.setInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES, 8);
BinaryExternalSorter sorter = new BinaryExternalSorter(
new Object(),
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorterTest.java
index 38b9469..dc2b404 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorterTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorterTest.java
@@ -70,9 +70,9 @@ public class BufferedKVExternalSorterTest {
int spillNumber, int recordNumberPerFile, boolean spillCompress) {
ioManager = new IOManagerAsync();
conf = new Configuration();
- conf.setInteger(ExecutionConfigOptions.SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM, 5);
+ conf.setInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES, 5);
if (!spillCompress) {
- conf.setBoolean(ExecutionConfigOptions.SQL_EXEC_SPILL_COMPRESSION_ENABLED, false);
+ conf.setBoolean(ExecutionConfigOptions.TABLE_EXEC_SPILL_COMPRESSION_ENABLED, false);
}
this.spillNumber = spillNumber;
this.recordNumberPerFile = recordNumberPerFile;