From 312fe4c1ce17ac6e5515bcab560b86456993daf5 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 24 Jun 2021 09:56:28 +0200 Subject: [PATCH] [FLINK-22880][table] Remove 'blink' term from code base This removes all mentionings of the term "blink" in the code base. In order to reduce user confusion, do not use this term anymore but refer to as "Flink SQL" or "Flink Table API". This closes #16374. --- .../connectors/table/hive/hive_catalog.md | 3 +- .../connectors/table/hive/hive_dialect.md | 1 - .../connectors/table/hive/hive_functions.md | 1 - .../docs/connectors/table/hive/overview.md | 5 --- docs/content.zh/docs/connectors/table/jdbc.md | 1 - .../dev/python/table/intro_to_table_api.md | 12 ++---- .../dev/python/table/table_environment.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 5 +-- .../table/udfs/vectorized_python_udfs.md | 2 - docs/content.zh/docs/dev/table/common.md | 6 +-- .../dev/table/concepts/versioned_tables.md | 1 - docs/content.zh/docs/dev/table/config.md | 2 - docs/content.zh/docs/dev/table/sql/set.md | 6 +-- .../docs/connectors/table/formats/raw.md | 2 +- .../connectors/table/hive/hive_catalog.md | 3 +- .../connectors/table/hive/hive_dialect.md | 5 +-- .../connectors/table/hive/hive_functions.md | 1 - .../docs/connectors/table/hive/overview.md | 8 ---- docs/content/docs/connectors/table/jdbc.md | 1 - .../dev/python/table/intro_to_table_api.md | 13 ++---- .../dev/python/table/table_environment.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 5 +-- .../table/udfs/vectorized_python_udfs.md | 2 - docs/content/docs/dev/table/common.md | 4 +- docs/content/docs/dev/table/sql/set.md | 6 +-- .../table/functions/hive/HiveGenericUDAF.java | 4 +- .../connectors/hive/HiveDialectITCase.java | 2 +- .../hive/HiveDialectQueryITCase.java | 3 +- .../connectors/hive/HiveLookupJoinITCase.java | 15 +++---- .../connectors/hive/HiveRunnerITCase.java | 8 ++-- .../connectors/hive/HiveTableSinkITCase.java | 17 ++++---- .../hive/HiveTableSourceITCase.java | 21 ++++------ .../hive/TableEnvHiveConnectorITCase.java | 3 +- .../HiveInputFormatPartitionReaderITCase.java | 3 +- .../table/catalog/hive/HiveCatalogITCase.java | 5 +-- ...kITCase.java => HiveCatalogUdfITCase.java} | 10 ++--- .../table/catalog/hive/HiveTestUtils.java | 14 +++---- .../table/module/hive/HiveModuleTest.java | 14 +++---- .../jdbc/dialect/AbstractDialect.java | 2 - .../jdbc/catalog/PostgresCatalogTestBase.java | 2 +- .../flink/util/CloseableIteratorTest.java | 2 +- .../flink-python-test/pom.xml | 2 +- .../flink-python-test/python/python_job.py | 3 +- ...fSqlJob.java => BatchPythonUdfSqlJob.java} | 4 +- ...SqlJob.java => StreamPythonUdfSqlJob.java} | 4 +- flink-end-to-end-tests/run-nightly-tests.sh | 4 +- .../test-data/tpch/modified-query/q15.sql | 2 +- .../test-data/tpch/modified-query/q20.sql | 2 +- .../test-data/tpch/modified-query/q6.sql | 2 +- .../test-scripts/test_pyflink.sh | 6 +-- .../test-scripts/test_tpch.sh | 1 - .../json/JsonRowDataSerDeSchemaTest.java | 8 ++-- flink-python/pyflink/table/table_config.py | 3 +- .../pyflink/table/table_environment.py | 3 +- flink-python/pyflink/table/tests/test_calc.py | 4 +- .../table/tests/test_column_operation.py | 4 +- .../pyflink/table/tests/test_correlate.py | 4 +- .../pyflink/table/tests/test_dependency.py | 8 ++-- .../pyflink/table/tests/test_distinct.py | 4 +- .../pyflink/table/tests/test_explain.py | 4 +- .../pyflink/table/tests/test_expression.py | 2 +- flink-python/pyflink/table/tests/test_join.py | 4 +- .../table/tests/test_pandas_conversion.py | 14 +++---- .../pyflink/table/tests/test_pandas_udaf.py | 8 ++-- .../pyflink/table/tests/test_pandas_udf.py | 14 +++---- .../table/tests/test_row_based_operation.py | 8 ++-- .../table/tests/test_schema_operation.py | 4 +- .../pyflink/table/tests/test_set_operation.py | 4 +- flink-python/pyflink/table/tests/test_sort.py | 4 +- flink-python/pyflink/table/tests/test_sql.py | 4 +- .../table/tests/test_table_environment_api.py | 6 +-- .../pyflink/table/tests/test_types.py | 4 +- flink-python/pyflink/table/tests/test_udaf.py | 4 +- flink-python/pyflink/table/tests/test_udf.py | 14 +++---- flink-python/pyflink/table/tests/test_udtf.py | 12 +++--- .../pyflink/table/tests/test_window.py | 8 ++-- flink-python/pyflink/table/types.py | 3 +- .../pyflink/testing/test_case_utils.py | 12 +++--- .../flink/table/runtime/arrow/ArrowUtils.java | 8 +--- ...AbstractPythonStreamAggregateOperator.java | 7 ++-- .../PythonStreamGroupAggregateOperator.java | 2 +- ...thonStreamGroupTableAggregateOperator.java | 2 +- ...honStreamGroupWindowAggregateOperator.java | 2 +- ...ctRowDataPythonScalarFunctionOperator.java | 2 +- .../RowDataPythonScalarFunctionOperator.java | 7 ++-- ...DataArrowPythonScalarFunctionOperator.java | 2 +- .../RowDataPythonTableFunctionOperator.java | 6 +-- .../runtime/typeutils/PythonTypeUtils.java | 6 +-- .../python/ArrayDataSerializer.java | 2 +- .../serializers/python/DateSerializer.java | 4 +- .../serializers/python/MapDataSerializer.java | 2 +- .../serializers/python/RowDataSerializer.java | 2 +- .../serializers/python/StringSerializer.java | 2 +- .../serializers/python/TimeSerializer.java | 4 +- .../python/TimestampSerializer.java | 5 +-- .../python/PythonFunctionFactoryTest.java | 41 ++++++++----------- .../typeutils/PythonTypeUtilsTest.java | 6 +-- .../factories/StreamTableSinkFactory.java | 3 +- .../factories/StreamTableSourceFactory.java | 3 +- .../table/sinks/AppendStreamTableSink.java | 3 +- .../table/sinks/OutputFormatTableSink.java | 3 +- .../table/sinks/RetractStreamTableSink.java | 3 +- .../flink/table/sinks/StreamTableSink.java | 3 +- .../table/sinks/UpsertStreamTableSink.java | 3 +- .../table/sources/InputFormatTableSource.java | 3 +- .../table/sources/StreamTableSource.java | 3 +- .../apache/flink/table/api/TableConfig.java | 6 ++- .../api/config/ExecutionConfigOptions.java | 2 - .../api/config/OptimizerConfigOptions.java | 2 - .../table/api/internal/StatementSetImpl.java | 2 - .../internal/TableEnvironmentInternal.java | 8 ---- .../flink/table/delegation/Planner.java | 6 --- .../table/sinks/OverwritableTableSink.java | 3 +- .../table/sinks/PartitionableTableSink.java | 4 +- .../apache/flink/table/sinks/TableSink.java | 3 +- .../table/sources/DefinedFieldMapping.java | 3 +- .../sources/DefinedProctimeAttribute.java | 4 +- .../sources/DefinedRowtimeAttributes.java | 4 +- .../flink/table/sources/FieldComputer.java | 4 +- .../table/sources/FilterableTableSource.java | 9 +--- .../table/sources/LimitableTableSource.java | 4 +- .../table/sources/LookupableTableSource.java | 3 +- .../NestedFieldsProjectableTableSource.java | 4 +- .../sources/PartitionableTableSource.java | 4 +- .../table/sources/ProjectableTableSource.java | 4 +- .../sources/RowtimeAttributeDescriptor.java | 4 +- .../flink/table/sources/TableSource.java | 3 +- .../tsextractors/TimestampExtractor.java | 4 +- .../LegacyTypeInfoDataTypeConverter.java | 2 +- flink-table/flink-table-planner-blink/pom.xml | 11 +++-- .../converter/CustomizedConvertRule.java | 2 +- .../converter/OverConvertRule.java | 2 +- .../RichTableSourceQueryOperation.java | 2 +- .../plan/FlinkCalciteCatalogReader.java | 2 +- .../planner/utils/InternalConfigOptions.java | 2 - .../planner/calcite/FlinkTypeFactory.scala | 2 +- .../planner/delegation/PlannerBase.scala | 2 +- .../parse/SetOperationParseStrategyTest.java | 12 ++++-- .../UserDefinedScalarFunctionTest.scala | 2 +- .../batch/sql/agg/AggregateITCaseBase.scala | 2 +- .../runtime/stream/sql/AggregateITCase.scala | 2 +- .../runtime/stream/sql/CorrelateITCase.scala | 2 +- .../stream/sql/MatchRecognizeITCase.scala | 4 +- flink-table/flink-table-runtime-blink/pom.xml | 6 +-- flink-table/flink-table-uber-blink/pom.xml | 5 +-- tools/azure-pipelines/jobs-template.yml | 4 +- tools/ci/stage.sh | 27 ++++++------ 147 files changed, 315 insertions(+), 432 deletions(-) rename flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/{HiveCatalogUseBlinkITCase.java => HiveCatalogUdfITCase.java} (97%) rename flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/{BlinkBatchPythonUdfSqlJob.java => BatchPythonUdfSqlJob.java} (97%) rename flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/{BlinkStreamPythonUdfSqlJob.java => StreamPythonUdfSqlJob.java} (94%) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_catalog.md b/docs/content.zh/docs/connectors/table/hive/hive_catalog.md index 0353a1be80bfc..e1be3da8bc673 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_catalog.md @@ -147,7 +147,6 @@ Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CL ```yaml execution: - planner: blink type: streaming ... current-catalog: myhive # set the HiveCatalog as the current catalog of the session @@ -394,4 +393,4 @@ Something to note about the type mapping: ## Scala Shell -NOTE: since blink planner is not well supported in Scala Shell at the moment, it's **NOT** recommended to use Hive connector in Scala Shell. +Note: It's **NOT** recommended to use the Hive connector in the Scala Shell. diff --git a/docs/content.zh/docs/connectors/table/hive/hive_dialect.md b/docs/content.zh/docs/connectors/table/hive/hive_dialect.md index 7e21ff365e194..3101f0db990aa 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_dialect.md @@ -40,7 +40,6 @@ SQL 方言可以通过 `table.sql-dialect` 属性指定。因此你可以通过 ```yaml execution: - planner: blink type: batch result-mode: table diff --git a/docs/content.zh/docs/connectors/table/hive/hive_functions.md b/docs/content.zh/docs/connectors/table/hive/hive_functions.md index 4d1d071b290ee..c505c7a305079 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_functions.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_functions.md @@ -97,7 +97,6 @@ To use a Hive User Defined Function, user have to - set a HiveCatalog backed by Hive Metastore that contains that function as current catalog of the session - include a jar that contains that function in Flink's classpath -- use Blink planner. ## Using Hive User Defined Functions diff --git a/docs/content.zh/docs/connectors/table/hive/overview.md b/docs/content.zh/docs/connectors/table/hive/overview.md index 3bd96aa86e4e2..3ce092e8992ae 100644 --- a/docs/content.zh/docs/connectors/table/hive/overview.md +++ b/docs/content.zh/docs/connectors/table/hive/overview.md @@ -39,8 +39,6 @@ Flink 与 Hive 的集成包含两个层面。 `HiveCatalog`的设计提供了与 Hive 良好的兼容性,用户可以"开箱即用"的访问其已有的 Hive 数仓。 您不需要修改现有的 Hive Metastore,也不需要更改表的数据位置或分区。 -* 我们强烈建议用户使用 [Blink planner]({{< ref "docs/dev/table/overview" >}}#dependency-structure) 与 Hive 集成。 - ## 支持的Hive版本 Flink 支持一下的 Hive 版本。 @@ -303,8 +301,6 @@ export HADOOP_CLASSPATH=`hadoop classpath` 通过 TableEnvironment 或者 YAML 配置,使用 [Catalog 接口]({{< ref "docs/dev/table/catalogs" >}}) 和 [HiveCatalog]({{< ref "docs/connectors/table/hive/hive_catalog" >}})连接到现有的 Hive 集群。 -请注意,虽然 HiveCatalog 不需要特定的 planner,但读写Hive表仅适用于 Blink planner。因此,强烈建议您在连接到 Hive 仓库时使用 Blink planner。 - 以下是如何连接到 Hive 的示例: {{< tabs "2ca7cad8-0b84-45db-92d9-a75abd8808e7" >}} @@ -367,7 +363,6 @@ tableEnv.use_catalog("myhive") ```yaml execution: - planner: blink ... current-catalog: myhive # set the HiveCatalog as the current catalog of the session current-database: mydatabase diff --git a/docs/content.zh/docs/connectors/table/jdbc.md b/docs/content.zh/docs/connectors/table/jdbc.md index ed76dc4ff2882..792cfbee88270 100644 --- a/docs/content.zh/docs/connectors/table/jdbc.md +++ b/docs/content.zh/docs/connectors/table/jdbc.md @@ -414,7 +414,6 @@ t_env.use_catalog("mypg") ```yaml execution: - planner: blink ... current-catalog: mypg # 设置 JdbcCatalog 为会话的当前 catalog current-database: mydb diff --git a/docs/content.zh/docs/dev/python/table/intro_to_table_api.md b/docs/content.zh/docs/dev/python/table/intro_to_table_api.md index da3c4ed190b26..fa3d4a734ab09 100644 --- a/docs/content.zh/docs/dev/python/table/intro_to_table_api.md +++ b/docs/content.zh/docs/dev/python/table/intro_to_table_api.md @@ -91,11 +91,11 @@ table_env.execute_sql("INSERT INTO print SELECT * FROM datagen").wait() ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink streaming TableEnvironment +# create a streaming TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) -# or create a blink batch TableEnvironment +# or create a batch TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) ``` @@ -112,10 +112,6 @@ table_env = TableEnvironment.create(env_settings) * 管理 Python 依赖,更多细节可查阅 [依赖管理]({{< ref "docs/dev/python/dependency_management" >}}) * 提交作业执行 -目前有2个可用的执行器 : flink 执行器 和 blink 执行器。 - -你应该在当前程序中显式地设置使用哪个执行器,建议尽可能使用 blink 执行器。 - {{< top >}} 创建表 @@ -132,7 +128,7 @@ table_env = TableEnvironment.create(env_settings) ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# 创建 blink 批 TableEnvironment +# 创建 批 TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) @@ -196,7 +192,7 @@ print('Now the type of the "id" column is %s.' % type) ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# 创建 blink 流 TableEnvironment +# 创建 流 TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) diff --git a/docs/content.zh/docs/dev/python/table/table_environment.md b/docs/content.zh/docs/dev/python/table/table_environment.md index cce9ac1056d4e..bbbad97461a86 100644 --- a/docs/content.zh/docs/dev/python/table/table_environment.md +++ b/docs/content.zh/docs/dev/python/table/table_environment.md @@ -50,7 +50,7 @@ table_env = TableEnvironment.create(env_settings) from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import StreamTableEnvironment -# create a blink streaming TableEnvironment from a StreamExecutionEnvironment +# create a streaming TableEnvironment from a StreamExecutionEnvironment env = StreamExecutionEnvironment.get_execution_environment() table_env = StreamTableEnvironment.create(env) ``` diff --git a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md index 503783c7900bb..124d1bbfcc7ef 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md @@ -235,7 +235,7 @@ def iterable_func(x): A user-defined aggregate function (_UDAGG_) maps scalar values of multiple rows to a new scalar value. -**NOTE:** Currently the general user-defined aggregate function is only supported in the GroupBy aggregation and Group Window Aggregation of the blink planner in streaming mode. For batch mode, it's currently not supported and it is recommended to use the [Vectorized Aggregate Functions]({{< ref "docs/dev/python/table/udfs/vectorized_python_udfs" >}}#vectorized-aggregate-functions). +**NOTE:** Currently the general user-defined aggregate function is only supported in the GroupBy aggregation and Group Window Aggregation in streaming mode. For batch mode, it's currently not supported and it is recommended to use the [Vectorized Aggregate Functions]({{< ref "docs/dev/python/table/udfs/vectorized_python_udfs" >}}#vectorized-aggregate-functions). The behavior of an aggregate function is centered around the concept of an accumulator. The _accumulator_ is an intermediate data structure that stores the aggregated values until a final aggregation result @@ -416,8 +416,7 @@ A user-defined table aggregate function (_UDTAGG_) maps scalar values of multipl The returned record may consist of one or more fields. If an output record consists of only a single field, the structured record can be omitted, and a scalar value can be emitted that will be implicitly wrapped into a row by the runtime. -**NOTE:** Currently the general user-defined table aggregate function is only supported in the GroupBy aggregation -of the blink planner in streaming mode. +**NOTE:** Currently the general user-defined table aggregate function is only supported in the GroupBy aggregation in streaming mode. Similar to an [aggregate function](#aggregate-functions), the behavior of a table aggregate is centered around the concept of an accumulator. The accumulator is an intermediate data structure that stores the aggregated values until a final aggregation result is computed. diff --git a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md index 6a0a7ec140622..af6c7f60c15b1 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -77,8 +77,6 @@ table_env.sql_query("SELECT add(bigint, bigint) FROM MyTable") 注意 向量化聚合函数不支持部分聚合,而且一个组或者窗口内的所有数据, 在执行的过程中,会被同时加载到内存,所以需要确保所配置的内存大小足够容纳这些数据。 -注意 向量化聚合函数只支持运行在 Blink Planner 上。 - 以下示例显示了如何定一个自己的向量化聚合函数,该函数计算一列的平均值,并在 `GroupBy Aggregation`, `GroupBy Window Aggregation` and `Over Window Aggregation` 使用它: diff --git a/docs/content.zh/docs/dev/table/common.md b/docs/content.zh/docs/dev/table/common.md index 409e1970a064a..84391703661cb 100644 --- a/docs/content.zh/docs/dev/table/common.md +++ b/docs/content.zh/docs/dev/table/common.md @@ -183,11 +183,11 @@ val tEnv = TableEnvironment.create(settings) ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink streaming TableEnvironment +# create a streaming TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) -# create a blink batch TableEnvironment +# create a batch TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) @@ -310,7 +310,7 @@ table_env.register_table("projectedTable", proj_table) **注意:** 从传统数据库系统的角度来看,`Table` 对象与 `VIEW` 视图非常像。也就是,定义了 `Table` 的查询是没有被优化的, 而且会被内嵌到另一个引用了这个注册了的 `Table`的查询中。如果多个查询都引用了同一个注册了的`Table`,那么它会被内嵌每个查询中并被执行多次, -也就是说注册了的`Table`的结果**不会**被共享(注:Blink 计划器的`TableEnvironment`会优化成只执行一次)。 +也就是说注册了的`Table`的结果**不会**被共享。 {{< top >}} diff --git a/docs/content.zh/docs/dev/table/concepts/versioned_tables.md b/docs/content.zh/docs/dev/table/concepts/versioned_tables.md index bfb6131fc4a71..77642ca30f803 100644 --- a/docs/content.zh/docs/dev/table/concepts/versioned_tables.md +++ b/docs/content.zh/docs/dev/table/concepts/versioned_tables.md @@ -115,7 +115,6 @@ Yen 1 时态表 ----- -注意 仅 Blink planner 支持此功能。 Flink 使用主键约束和事件时间来定义一张版本表和版本视图。 diff --git a/docs/content.zh/docs/dev/table/config.md b/docs/content.zh/docs/dev/table/config.md index a313bdc306083..c4ed51966a70b 100644 --- a/docs/content.zh/docs/dev/table/config.md +++ b/docs/content.zh/docs/dev/table/config.md @@ -91,8 +91,6 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000'; {{< /tab >}} {{< /tabs >}} -注意 目前,key-value 配置项仅被 Blink planner 支持。 - ### 执行配置 以下选项可用于优化查询执行的性能。 diff --git a/docs/content.zh/docs/dev/table/sql/set.md b/docs/content.zh/docs/dev/table/sql/set.md index 078f4efbf28a9..9577716911969 100644 --- a/docs/content.zh/docs/dev/table/sql/set.md +++ b/docs/content.zh/docs/dev/table/sql/set.md @@ -43,11 +43,11 @@ The following examples show how to run a `SET` statement in SQL CLI. {{< tabs "set" >}} {{< tab "SQL CLI" >}} ```sql -Flink SQL> SET 'table.planner' = 'blink'; +Flink SQL> SET 'table.local-time-zone' = 'Europe/Berlin'; [INFO] Session property has been set. Flink SQL> SET; -'table.planner' = 'blink' +'table.local-time-zone' = 'Europe/Berlin' ``` {{< /tab >}} {{< /tabs >}} @@ -58,6 +58,6 @@ Flink SQL> SET; SET ('key' = 'value')? ``` -If no key and value are specified, it just print all the properties. Otherwise, set the key with specified value. +If no key and value are specified, it just prints all the properties. Otherwise, set the key with specified value. {{< top >}} diff --git a/docs/content/docs/connectors/table/formats/raw.md b/docs/content/docs/connectors/table/formats/raw.md index 8bb5f57efa03b..4140063d8bc0e 100644 --- a/docs/content/docs/connectors/table/formats/raw.md +++ b/docs/content/docs/connectors/table/formats/raw.md @@ -33,7 +33,7 @@ The Raw format allows to read and write raw (byte based) values as a single colu Note: this format encodes `null` values as `null` of `byte[]` type. This may have limitation when used in `upsert-kafka`, because `upsert-kafka` treats `null` values as a tombstone message (DELETE on the key). Therefore, we recommend avoiding using `upsert-kafka` connector and the `raw` format as a `value.format` if the field can have a `null` value. -The Raw connector is built-in into the Blink planner, no additional dependencies are required. +The Raw connector is built-in, no additional dependencies are required. Example ---------------- diff --git a/docs/content/docs/connectors/table/hive/hive_catalog.md b/docs/content/docs/connectors/table/hive/hive_catalog.md index e5a8cc815a461..f02c4f6f3ca17 100644 --- a/docs/content/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content/docs/connectors/table/hive/hive_catalog.md @@ -147,7 +147,6 @@ Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CL ```yaml execution: - planner: blink type: streaming ... current-catalog: myhive # set the HiveCatalog as the current catalog of the session @@ -394,4 +393,4 @@ Something to note about the type mapping: ## Scala Shell -NOTE: since blink planner is not well supported in Scala Shell at the moment, it's **NOT** recommended to use Hive connector in Scala Shell. +Note: It's **NOT** recommended to use Hive connector in Scala Shell. diff --git a/docs/content/docs/connectors/table/hive/hive_dialect.md b/docs/content/docs/connectors/table/hive/hive_dialect.md index 1fcaa3f91f016..38c0845b2e624 100644 --- a/docs/content/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content/docs/connectors/table/hive/hive_dialect.md @@ -46,7 +46,6 @@ the `configuration` section of the yaml file for your SQL Client. ```yaml execution: - planner: blink type: batch result-mode: table @@ -59,10 +58,10 @@ You can also set the dialect after the SQL Client has launched. ```bash -Flink SQL> set table.sql-dialect=hive; -- to use hive dialect +Flink SQL> SET 'table.sql-dialect' = 'hive'; -- to use hive dialect [INFO] Session property has been set. -Flink SQL> set table.sql-dialect=default; -- to use default dialect +Flink SQL> SET 'table.sql-dialect' = 'default'; -- to use default dialect [INFO] Session property has been set. ``` diff --git a/docs/content/docs/connectors/table/hive/hive_functions.md b/docs/content/docs/connectors/table/hive/hive_functions.md index 0805970f58c1f..a9dcfe61557fc 100644 --- a/docs/content/docs/connectors/table/hive/hive_functions.md +++ b/docs/content/docs/connectors/table/hive/hive_functions.md @@ -97,7 +97,6 @@ To use a Hive User Defined Function, user have to - set a HiveCatalog backed by Hive Metastore that contains that function as current catalog of the session - include a jar that contains that function in Flink's classpath -- use Blink planner. ## Using Hive User Defined Functions diff --git a/docs/content/docs/connectors/table/hive/overview.md b/docs/content/docs/connectors/table/hive/overview.md index 1002cfe1b9056..148eefefe09f2 100644 --- a/docs/content/docs/connectors/table/hive/overview.md +++ b/docs/content/docs/connectors/table/hive/overview.md @@ -39,10 +39,6 @@ The second is to offer Flink as an alternative engine for reading and writing Hi The `HiveCatalog` is designed to be “out of the box” compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables. -* Note that we highly recommend users using the [blink planner]({{< ref "docs/dev/table/overview" >}}#dependency-structure) with Hive integration. - - - ## Supported Hive Versions Flink supports the following Hive versions. @@ -309,9 +305,6 @@ You're supposed to add dependencies as stated above at runtime. Connect to an existing Hive installation using the [catalog interface]({{< ref "docs/dev/table/catalogs" >}}) and [HiveCatalog]({{< ref "docs/connectors/table/hive/hive_catalog" >}}) through the table environment or YAML configuration. -Please note while HiveCatalog doesn't require a particular planner, reading/writing Hive tables only works with blink planner. -Therefore it's highly recommended that you use blink planner when connecting to your Hive warehouse. - Following is an example of how to connect to Hive: {{< tabs "5d3cc7e1-a304-4f9e-b36e-ff1f32394ec7" >}} @@ -374,7 +367,6 @@ tableEnv.use_catalog("myhive") ```yaml execution: - planner: blink ... current-catalog: myhive # set the HiveCatalog as the current catalog of the session current-database: mydatabase diff --git a/docs/content/docs/connectors/table/jdbc.md b/docs/content/docs/connectors/table/jdbc.md index 9d3303c32bd54..a6635d12a57f8 100644 --- a/docs/content/docs/connectors/table/jdbc.md +++ b/docs/content/docs/connectors/table/jdbc.md @@ -413,7 +413,6 @@ t_env.use_catalog("mypg") ```yaml execution: - planner: blink ... current-catalog: mypg # set the JdbcCatalog as the current catalog of the session current-database: mydb diff --git a/docs/content/docs/dev/python/table/intro_to_table_api.md b/docs/content/docs/dev/python/table/intro_to_table_api.md index f09a9cd99a80f..223407353a85e 100644 --- a/docs/content/docs/dev/python/table/intro_to_table_api.md +++ b/docs/content/docs/dev/python/table/intro_to_table_api.md @@ -89,11 +89,11 @@ The `TableEnvironment` is a central concept of the Table API and SQL integration ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink streaming TableEnvironment +# create a streaming TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) -# or create a blink batch TableEnvironment +# or create a batch TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) ``` @@ -110,11 +110,6 @@ The `TableEnvironment` is responsible for: * Managing Python dependencies, see [Dependency Management]({{< ref "docs/dev/python/dependency_management" >}}) for more details * Submitting the jobs for execution -Currently there are 2 planners available: flink planner and blink planner. - -You should explicitly set which planner to use in the current program. -We recommend using the blink planner as much as possible. - {{< top >}} Create Tables @@ -131,7 +126,7 @@ You can create a Table from a list object: ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink batch TableEnvironment +# create a batch TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) @@ -195,7 +190,7 @@ You can create a Table using connector DDL: ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink stream TableEnvironment +# create a stream TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) diff --git a/docs/content/docs/dev/python/table/table_environment.md b/docs/content/docs/dev/python/table/table_environment.md index 4618d52cbb2b2..e2f055fbfe824 100644 --- a/docs/content/docs/dev/python/table/table_environment.md +++ b/docs/content/docs/dev/python/table/table_environment.md @@ -51,7 +51,7 @@ Alternatively, users can create a `StreamTableEnvironment` from an existing `Str from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import StreamTableEnvironment -# create a blink streaming TableEnvironment from a StreamExecutionEnvironment +# create a streaming TableEnvironment from a StreamExecutionEnvironment env = StreamExecutionEnvironment.get_execution_environment() table_env = StreamTableEnvironment.create(env) ``` diff --git a/docs/content/docs/dev/python/table/udfs/python_udfs.md b/docs/content/docs/dev/python/table/udfs/python_udfs.md index cf8a3a50d3526..745124b7bc99b 100644 --- a/docs/content/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/python_udfs.md @@ -235,7 +235,7 @@ def iterable_func(x): A user-defined aggregate function (_UDAGG_) maps scalar values of multiple rows to a new scalar value. -**NOTE:** Currently the general user-defined aggregate function is only supported in the GroupBy aggregation and Group Window Aggregation of the blink planner in streaming mode. For batch mode, it's currently not supported and it is recommended to use the [Vectorized Aggregate Functions]({{< ref "docs/dev/python/table/udfs/vectorized_python_udfs" >}}#vectorized-aggregate-functions). +**NOTE:** Currently the general user-defined aggregate function is only supported in the GroupBy aggregation and Group Window Aggregation in streaming mode. For batch mode, it's currently not supported and it is recommended to use the [Vectorized Aggregate Functions]({{< ref "docs/dev/python/table/udfs/vectorized_python_udfs" >}}#vectorized-aggregate-functions). The behavior of an aggregate function is centered around the concept of an accumulator. The _accumulator_ is an intermediate data structure that stores the aggregated values until a final aggregation result @@ -417,8 +417,7 @@ A user-defined table aggregate function (_UDTAGG_) maps scalar values of multipl The returned record may consist of one or more fields. If an output record consists of only a single field, the structured record can be omitted, and a scalar value can be emitted that will be implicitly wrapped into a row by the runtime. -**NOTE:** Currently the general user-defined table aggregate function is only supported in the GroupBy aggregation -of the blink planner in streaming mode. +**NOTE:** Currently the general user-defined table aggregate function is only supported in the GroupBy aggregation in streaming mode. Similar to an [aggregate function](#aggregate-functions), the behavior of a table aggregate is centered around the concept of an accumulator. The accumulator is an intermediate data structure that stores the aggregated values until a final aggregation result is computed. diff --git a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md index cd30626b1198d..7b79eabab8eff 100644 --- a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -76,8 +76,6 @@ to [the relevant documentation]({{< ref "docs/dev/table/tableApi" >}}?code_tab=p Note Pandas UDAF does not support partial aggregation. Besides, all the data for a group or window will be loaded into memory at the same time during execution and so you must make sure that the data of a group or window could fit into the memory. -Note Pandas UDAF is only supported in Blink Planner. - The following example shows how to define your own vectorized Python aggregate function which computes mean, and use it in `GroupBy Aggregation`, `GroupBy Window Aggregation` and `Over Window Aggregation`: diff --git a/docs/content/docs/dev/table/common.md b/docs/content/docs/dev/table/common.md index fb5d36bb79ea9..ee8d75039aa79 100644 --- a/docs/content/docs/dev/table/common.md +++ b/docs/content/docs/dev/table/common.md @@ -183,11 +183,11 @@ val tEnv = TableEnvironment.create(settings) ```python from pyflink.table import EnvironmentSettings, TableEnvironment -# create a blink streaming TableEnvironment +# create a streaming TableEnvironment env_settings = EnvironmentSettings.in_streaming_mode() table_env = TableEnvironment.create(env_settings) -# create a blink batch TableEnvironment +# create a batch TableEnvironment env_settings = EnvironmentSettings.in_batch_mode() table_env = TableEnvironment.create(env_settings) diff --git a/docs/content/docs/dev/table/sql/set.md b/docs/content/docs/dev/table/sql/set.md index 9768b52a5dd74..1e1eb0f3b57ca 100644 --- a/docs/content/docs/dev/table/sql/set.md +++ b/docs/content/docs/dev/table/sql/set.md @@ -43,11 +43,11 @@ The following examples show how to run a `SET` statement in SQL CLI. {{< tabs "set" >}} {{< tab "SQL CLI" >}} ```sql -Flink SQL> SET 'table.planner' = 'blink'; +Flink SQL> SET 'table.local-time-zone' = 'Europe/Berlin'; [INFO] Session property has been set. Flink SQL> SET; -'table.planner' = 'blink' +'table.local-time-zone' = 'Europe/Berlin' ``` {{< /tab >}} {{< /tabs >}} @@ -58,6 +58,6 @@ Flink SQL> SET; SET ('key' = 'value')? ``` -If no key and value are specified, it just print all the properties. Otherwise, set the key with specified value. +If no key and value are specified, it just prints all the properties. Otherwise, set the key with specified value. {{< top >}} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java index 7c100581c3480..99eb3b79c2efb 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java @@ -151,9 +151,9 @@ public GenericUDAFEvaluator createEvaluator(ObjectInspector[] inputInspectors) } /** - * This is invoked without calling open() in Blink, so we need to call init() for + * This is invoked without calling open(), so we need to call init() for * getNewAggregationBuffer(). TODO: re-evaluate how this will fit into Flink's new type - * inference and udf systemß + * inference and udf system */ @Override public GenericUDAFEvaluator.AggregationBuffer createAccumulator() { diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 1dbc40fa02705..d2e57000d4e28 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -102,7 +102,7 @@ public void setup() { HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES, false); hiveCatalog.open(); warehouse = hiveCatalog.getHiveConf().getVar(HiveConf.ConfVars.METASTOREWAREHOUSE); - tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java index a762ffd8b0775..389700057683c 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java @@ -388,8 +388,7 @@ private void runQuery(String query) throws Exception { } private static TableEnvironment getTableEnvWithHiveCatalog() { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); // automatically load hive module in hive-compatible mode diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java index d6d74072a3364..2ed075c5169ae 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java @@ -170,8 +170,7 @@ public void testLookupOptions() throws Exception { @Test public void testPartitionFetcherAndReader() throws Exception { // constructs test data using dynamic partition - TableEnvironment batchEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment batchEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); batchEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchEnv.useCatalog(hiveCatalog.getName()); batchEnv.executeSql( @@ -232,8 +231,7 @@ public void testLookupJoinBoundedTable() throws Exception { @Test public void testLookupJoinBoundedPartitionedTable() throws Exception { // constructs test data using dynamic partition - TableEnvironment batchEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment batchEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); batchEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchEnv.useCatalog(hiveCatalog.getName()); batchEnv.executeSql( @@ -259,8 +257,7 @@ public void testLookupJoinBoundedPartitionedTable() throws Exception { @Test public void testLookupJoinPartitionedTable() throws Exception { // constructs test data using dynamic partition - TableEnvironment batchEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment batchEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); batchEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchEnv.useCatalog(hiveCatalog.getName()); batchEnv.executeSql( @@ -290,8 +287,7 @@ public void testLookupJoinPartitionedTable() throws Exception { @Test public void testLookupJoinPartitionedTableWithPartitionTime() throws Exception { // constructs test data using dynamic partition - TableEnvironment batchEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment batchEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); batchEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchEnv.useCatalog(hiveCatalog.getName()); batchEnv.executeSql( @@ -320,8 +316,7 @@ public void testLookupJoinPartitionedTableWithPartitionTime() throws Exception { @Test public void testLookupJoinPartitionedTableWithCreateTime() throws Exception { // constructs test data using dynamic partition - TableEnvironment batchEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment batchEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); batchEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchEnv.useCatalog(hiveCatalog.getName()); batchEnv.executeSql( diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java index 01864aeb6bd55..9ee6681c9b140 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java @@ -209,8 +209,7 @@ public void testWriteNestedComplexType() throws Exception { @Test public void testWriteNullValues() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql("create database db1"); @@ -617,8 +616,7 @@ private void testCompressTextTable(boolean batch) throws Exception { } private static TableEnvironment getTableEnvWithHiveCatalog() { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); return tableEnv; @@ -627,7 +625,7 @@ private static TableEnvironment getTableEnvWithHiveCatalog() { private TableEnvironment getStreamTableEnvWithHiveCatalog() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); return tableEnv; diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java index be8453b53b58f..4ae3a079a1b49 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java @@ -86,8 +86,7 @@ public static void closeCatalog() { @Test public void testHiveTableSinkWithParallelismInBatch() { - final TableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + final TableEnvironment tEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); testHiveTableSinkWithParallelismBase( tEnv, "/explain/testHiveTableSinkWithParallelismInBatch.out"); } @@ -96,7 +95,7 @@ public void testHiveTableSinkWithParallelismInBatch() { public void testHiveTableSinkWithParallelismInStreaming() { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final TableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); testHiveTableSinkWithParallelismBase( tEnv, "/explain/testHiveTableSinkWithParallelismInStreaming.out"); } @@ -131,8 +130,7 @@ private void testHiveTableSinkWithParallelismBase( @Test public void testBatchAppend() throws Exception { - TableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tEnv.useCatalog(hiveCatalog.getName()); tEnv.executeSql("create database db1"); @@ -202,8 +200,7 @@ public void testStreamingAppend() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env); + StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvInStreamingMode(env); tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tEnv.useCatalog(hiveCatalog.getName()); @@ -237,7 +234,7 @@ public void testStreamingSinkWithTimestampLtzWatermark() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); env.enableCheckpointing(100); - StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env); + StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvInStreamingMode(env); tEnv.getConfig().setLocalTimeZone(ZoneId.of("Asia/Shanghai")); tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); @@ -405,7 +402,7 @@ private void testStreamingWrite( env.setParallelism(1); env.enableCheckpointing(100); - StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env); + StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvInStreamingMode(env); tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tEnv.useCatalog(hiveCatalog.getName()); tEnv.getConfig().setSqlDialect(SqlDialect.HIVE); @@ -498,7 +495,7 @@ private void testStreamingWrite( private void assertBatch(String table, List expected) { // using batch table env to query. List results = new ArrayList<>(); - TableEnvironment batchTEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment batchTEnv = HiveTestUtils.createTableEnvInBatchMode(); batchTEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); batchTEnv.useCatalog(hiveCatalog.getName()); batchTEnv diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java index 2102168875a02..45e2d0b138bbf 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java @@ -225,8 +225,7 @@ public void testPartitionPrunning() throws Exception { @Test public void testPartitionFilter() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); TestPartitionFilterCatalog catalog = new TestPartitionFilterCatalog( hiveCatalog.getName(), @@ -326,8 +325,7 @@ public void testPartitionFilter() throws Exception { @Test public void testPartitionFilterDateTimestamp() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); TestPartitionFilterCatalog catalog = new TestPartitionFilterCatalog( hiveCatalog.getName(), @@ -594,7 +592,7 @@ public void testStreamPartitionReadByPartitionName() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(100); StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tEnv.registerCatalog(catalogName, hiveCatalog); tEnv.useCatalog(catalogName); tEnv.executeSql( @@ -649,7 +647,7 @@ public void testStreamPartitionReadByCreateTime() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(100); StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tEnv.registerCatalog(catalogName, hiveCatalog); tEnv.useCatalog(catalogName); tEnv.executeSql( @@ -704,7 +702,7 @@ public void testStreamPartitionReadByPartitionTime() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(100); StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tEnv.registerCatalog(catalogName, hiveCatalog); tEnv.useCatalog(catalogName); tEnv.executeSql( @@ -773,7 +771,7 @@ private void testNonPartitionStreamingSource(Boolean useMapredReader, String tbl final String catalogName = "hive"; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tEnv.getConfig() .getConfiguration() .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, useMapredReader); @@ -838,7 +836,7 @@ private void testSourceConfig(boolean fallbackMR, boolean inferParallelism) thro HiveCatalog catalogSpy = spy(hiveCatalog); doReturn(Optional.of(tableFactorySpy)).when(catalogSpy).getTableFactory(); - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.getConfig() .getConfiguration() .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, fallbackMR); @@ -893,7 +891,7 @@ public void testStreamReadWithProjectPushDown() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(100); StreamTableEnvironment tEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.HIVE); + HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE); tEnv.registerCatalog(catalogName, hiveCatalog); tEnv.useCatalog(catalogName); tEnv.executeSql( @@ -940,8 +938,7 @@ public void testStreamReadWithProjectPushDown() throws Exception { } private static TableEnvironment createTableEnv() { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog("hive", hiveCatalog); tableEnv.useCatalog("hive"); return tableEnv; diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java index 28c3559fe2646..4f348546c74f1 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java @@ -558,8 +558,7 @@ public void testReadEmptyCollectionFromParquet() throws Exception { } private TableEnvironment getTableEnvWithHiveCatalog() { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); return tableEnv; diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java index 3c9517b6c5700..3b30c566a00d1 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java @@ -43,8 +43,7 @@ public class HiveInputFormatPartitionReaderITCase { @Test public void testReadMultipleSplits() throws Exception { HiveCatalog hiveCatalog = HiveTestUtils.createHiveCatalog(); - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java index 221c6363ae980..88c5eead674d0 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java @@ -482,7 +482,7 @@ public void testTemporaryGenericTable() throws Exception { @Test public void testCreateTableLike() throws Exception { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql("create table generic_table (x int) with ('connector'='COLLECTION')"); @@ -504,8 +504,7 @@ public void testCreateTableLike() throws Exception { @Test public void testViewSchema() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.DEFAULT); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.DEFAULT); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUdfITCase.java similarity index 97% rename from flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java rename to flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUdfITCase.java index a686492567e42..4d272156a9862 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUdfITCase.java @@ -75,7 +75,7 @@ * IT case for HiveCatalog. TODO: move to flink-connector-hive-test end-to-end test module once it's * setup */ -public class HiveCatalogUseBlinkITCase extends AbstractTestBase { +public class HiveCatalogUdfITCase extends AbstractTestBase { @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -98,7 +98,7 @@ public static void closeCatalog() { } @Test - public void testBlinkUdf() throws Exception { + public void testFlinkUdf() throws Exception { TableSchema schema = TableSchema.builder() .field("name", DataTypes.STRING()) @@ -247,8 +247,7 @@ private void testUdf(boolean batch) throws Exception { @Test public void testTimestampUDF() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql( @@ -275,8 +274,7 @@ public void testTimestampUDF() throws Exception { @Test public void testDateUDF() throws Exception { - TableEnvironment tableEnv = - HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql( 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 79a32e6f14ea1..b3717ae11cc9c 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 @@ -147,11 +147,11 @@ public static int getFreePort() throws IOException { throw new RuntimeException("Exhausted all ephemeral ports and didn't find a free one"); } - public static TableEnvironment createTableEnvWithBlinkPlannerBatchMode() { - return createTableEnvWithBlinkPlannerBatchMode(SqlDialect.DEFAULT); + public static TableEnvironment createTableEnvInBatchMode() { + return createTableEnvInBatchMode(SqlDialect.DEFAULT); } - public static TableEnvironment createTableEnvWithBlinkPlannerBatchMode(SqlDialect dialect) { + public static TableEnvironment createTableEnvInBatchMode(SqlDialect dialect) { TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); tableEnv.getConfig() .getConfiguration() @@ -160,12 +160,12 @@ public static TableEnvironment createTableEnvWithBlinkPlannerBatchMode(SqlDialec return tableEnv; } - public static StreamTableEnvironment createTableEnvWithBlinkPlannerStreamMode( + public static StreamTableEnvironment createTableEnvInStreamingMode( StreamExecutionEnvironment env) { - return createTableEnvWithBlinkPlannerStreamMode(env, SqlDialect.DEFAULT); + return createTableEnvInStreamingMode(env, SqlDialect.DEFAULT); } - public static StreamTableEnvironment createTableEnvWithBlinkPlannerStreamMode( + public static StreamTableEnvironment createTableEnvInStreamingMode( StreamExecutionEnvironment env, SqlDialect dialect) { StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); tableEnv.getConfig() @@ -176,7 +176,7 @@ public static StreamTableEnvironment createTableEnvWithBlinkPlannerStreamMode( } public static TableEnvironment createTableEnvWithHiveCatalog(HiveCatalog catalog) { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.registerCatalog(catalog.getName(), catalog); tableEnv.useCatalog(catalog.getName()); return tableEnv; diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java index d1cb87c43ba0f..427b5d8b69ec8 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java @@ -60,7 +60,7 @@ public void testNumberOfBuiltinFunctions() { verifyNumBuiltInFunctions(hiveVersion, hiveModule); // creating functions shouldn't change the number of built in functions - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.executeSql("create function myudf as 'org.apache.hadoop.hive.ql.udf.UDFPI'"); tableEnv.executeSql( "create function mygenericudf as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'"); @@ -130,7 +130,7 @@ public void testNonExistFunction() { @Test public void testConstantArguments() { - TableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tEnv = HiveTestUtils.createTableEnvInBatchMode(); tEnv.unloadModule("core"); tEnv.loadModule("hive", new HiveModule()); @@ -172,7 +172,7 @@ public void testConstantArguments() { @Test public void testDecimalReturnType() { - TableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tEnv = HiveTestUtils.createTableEnvInBatchMode(); tEnv.unloadModule("core"); tEnv.loadModule("hive", new HiveModule()); @@ -195,7 +195,7 @@ public void testBlackList() { @Test public void testConstantReturnValue() { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.unloadModule("core"); tableEnv.loadModule("hive", new HiveModule()); @@ -211,7 +211,7 @@ public void testConstantReturnValue() { @Test public void testEmptyStringLiteralParameters() { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.unloadModule("core"); tableEnv.loadModule("hive", new HiveModule()); @@ -233,7 +233,7 @@ public void testEmptyStringLiteralParameters() { @Test public void testFunctionsNeedSessionState() { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.unloadModule("core"); tableEnv.loadModule("hive", new HiveModule()); @@ -249,7 +249,7 @@ public void testFunctionsNeedSessionState() { @Test public void testCallUDFWithNoParam() { - TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode(); tableEnv.unloadModule("core"); tableEnv.loadModule("hive", new HiveModule()); diff --git a/flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/AbstractDialect.java b/flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/AbstractDialect.java index a75c3e5a854f0..2ca0977149618 100644 --- a/flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/AbstractDialect.java +++ b/flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/AbstractDialect.java @@ -50,7 +50,6 @@ public void validate(TableSchema schema) throws ValidationException { dialectName(), dt.toString())); } - // only validate precision of DECIMAL type for blink planner if (dt.getLogicalType() instanceof DecimalType) { int precision = ((DecimalType) dt.getLogicalType()).getPrecision(); if (precision > maxDecimalPrecision() || precision < minDecimalPrecision()) { @@ -65,7 +64,6 @@ public void validate(TableSchema schema) throws ValidationException { } } - // only validate precision of DECIMAL type for blink planner if (dt.getLogicalType() instanceof TimestampType) { int precision = ((TimestampType) dt.getLogicalType()).getPrecision(); if (precision > maxTimestampPrecision() || precision < minTimestampPrecision()) { diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java index ddccc9ba1c974..93c71fd38f9e3 100644 --- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java +++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java @@ -260,7 +260,7 @@ public static TestTable getPrimitiveTable(String primaryKeyName) { + "500"); } - // TODO: add back timestamptz once blink planner supports timestamp with timezone + // TODO: add back timestamptz once planner supports timestamp with timezone public static TestTable getArrayTable() { return new TestTable( TableSchema.builder() diff --git a/flink-core/src/test/java/org/apache/flink/util/CloseableIteratorTest.java b/flink-core/src/test/java/org/apache/flink/util/CloseableIteratorTest.java index 6c76c48cd6347..ed1f7a7be5f13 100644 --- a/flink-core/src/test/java/org/apache/flink/util/CloseableIteratorTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/CloseableIteratorTest.java @@ -30,7 +30,7 @@ @SuppressWarnings("unchecked") public class CloseableIteratorTest { - private static final String[] ELEMENTS = new String[] {"flink", "blink"}; + private static final String[] ELEMENTS = new String[] {"element-1", "element-2"}; @Test public void testFlattenEmpty() throws Exception { diff --git a/flink-end-to-end-tests/flink-python-test/pom.xml b/flink-end-to-end-tests/flink-python-test/pom.xml index 438a562995a5d..4922be559c97c 100644 --- a/flink-end-to-end-tests/flink-python-test/pom.xml +++ b/flink-end-to-end-tests/flink-python-test/pom.xml @@ -78,7 +78,7 @@ PythonUdfSqlJobExample - org.apache.flink.python.tests.BlinkStreamPythonUdfSqlJob + org.apache.flink.python.tests.StreamPythonUdfSqlJob diff --git a/flink-end-to-end-tests/flink-python-test/python/python_job.py b/flink-end-to-end-tests/flink-python-test/python/python_job.py index 2afcf193fd9c4..2852bc583dbef 100644 --- a/flink-end-to-end-tests/flink-python-test/python/python_job.py +++ b/flink-end-to-end-tests/flink-python-test/python/python_job.py @@ -33,8 +33,7 @@ def word_count(): "License you may not use this file except in compliance " \ "with the License" - env_settings = EnvironmentSettings.new_instance().in_batch_mode().use_blink_planner().build() - t_env = TableEnvironment.create(environment_settings=env_settings) + t_env = TableEnvironment.create(EnvironmentSettings.in_batch_mode()) # used to test pipeline.jars and pipleline.classpaths config_key = sys.argv[1] diff --git a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkBatchPythonUdfSqlJob.java b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java similarity index 97% rename from flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkBatchPythonUdfSqlJob.java rename to flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java index 7bbfba0c67624..5639acaffa8ea 100644 --- a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkBatchPythonUdfSqlJob.java +++ b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java @@ -27,8 +27,8 @@ import java.util.Iterator; import java.util.List; -/** A simple job used to test submitting the Python UDF job in blink batch mode. */ -public class BlinkBatchPythonUdfSqlJob { +/** A simple job used to test submitting the Python UDF job in batch mode. */ +public class BatchPythonUdfSqlJob { public static void main(String[] args) { TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); diff --git a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkStreamPythonUdfSqlJob.java b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/StreamPythonUdfSqlJob.java similarity index 94% rename from flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkStreamPythonUdfSqlJob.java rename to flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/StreamPythonUdfSqlJob.java index eef012154f8fb..eb61ce8c117b4 100644 --- a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BlinkStreamPythonUdfSqlJob.java +++ b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/StreamPythonUdfSqlJob.java @@ -26,8 +26,8 @@ import java.util.Iterator; import java.util.List; -/** A simple job used to test submitting the Python UDF job in blink stream mode. */ -public class BlinkStreamPythonUdfSqlJob { +/** A simple job used to test submitting the Python UDF job in stream mode. */ +public class StreamPythonUdfSqlJob { public static void main(String[] args) { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 426fa8a5e3c9c..f03ecff2fd96d 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -214,8 +214,8 @@ run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scrip run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" -run_test "TPC-H end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_tpch.sh" -run_test "TPC-DS end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_tpcds.sh" +run_test "TPC-H end-to-end test" "$END_TO_END_DIR/test-scripts/test_tpch.sh" +run_test "TPC-DS end-to-end test" "$END_TO_END_DIR/test-scripts/test_tpcds.sh" run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" "skip_check_exceptions" diff --git a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q15.sql b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q15.sql index 8182b3e3c6be6..769a74bc1ce6f 100644 --- a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q15.sql +++ b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q15.sql @@ -29,7 +29,7 @@ -- ) -- ORDER BY -- s_suppkey; --- Blink does not support view +-- we don't support view SELECT s_suppkey, diff --git a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q20.sql b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q20.sql index 744539845680b..fd256d893a18c 100644 --- a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q20.sql +++ b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q20.sql @@ -26,7 +26,7 @@ WHERE AND l_suppkey = ps_suppkey -- AND l_shipdate >= date('1994-01-01') -- AND l_shipdate < date('1994-01-01') + interval '1' YEAR - -- Blink does not support the above format + -- we don't support the above format AND l_shipdate >= date '1994-01-01' AND l_shipdate < date '1994-01-01' + interval '1' YEAR ) diff --git a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q6.sql b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q6.sql index 28fb52ed8c8e2..2b47a4dc0ff93 100644 --- a/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q6.sql +++ b/flink-end-to-end-tests/test-scripts/test-data/tpch/modified-query/q6.sql @@ -6,6 +6,6 @@ WHERE l_shipdate >= DATE '1994-01-01' AND l_shipdate < DATE '1994-01-01' + INTERVAL '1' YEAR -- AND l_discount BETWEEN decimal '0.06' - decimal '0.01' AND decimal '0.06' + decimal '0.01' --- Blink currently does not support the above feature +-- we don't support the above feature AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 AND l_quantity < 24 diff --git a/flink-end-to-end-tests/test-scripts/test_pyflink.sh b/flink-end-to-end-tests/test-scripts/test_pyflink.sh index 27cfd0ac53312..968f9de14368d 100755 --- a/flink-end-to-end-tests/test-scripts/test_pyflink.sh +++ b/flink-end-to-end-tests/test-scripts/test_pyflink.sh @@ -136,7 +136,7 @@ PYFLINK_CLIENT_EXECUTABLE=${PYTHON_EXEC} "${FLINK_DIR}/bin/flink" run \ -py "${FLINK_PYTHON_TEST_DIR}/python/python_job.py" \ pipeline.classpaths "file://${FLINK_PYTHON_TEST_DIR}/target/PythonUdfSqlJobExample.jar" -echo "Test blink stream python udf sql job:\n" +echo "Test stream python udf sql job:\n" PYFLINK_CLIENT_EXECUTABLE=${PYTHON_EXEC} "${FLINK_DIR}/bin/flink" run \ -p 2 \ -pyfs "${FLINK_PYTHON_TEST_DIR}/python/add_one.py" \ @@ -145,14 +145,14 @@ PYFLINK_CLIENT_EXECUTABLE=${PYTHON_EXEC} "${FLINK_DIR}/bin/flink" run \ -pyexec "venv.zip/.conda/bin/python" \ "${FLINK_PYTHON_TEST_DIR}/target/PythonUdfSqlJobExample.jar" -echo "Test blink batch python udf sql job:\n" +echo "Test batch python udf sql job:\n" PYFLINK_CLIENT_EXECUTABLE=${PYTHON_EXEC} "${FLINK_DIR}/bin/flink" run \ -p 2 \ -pyfs "${FLINK_PYTHON_TEST_DIR}/python/add_one.py" \ -pyreq "${REQUIREMENTS_PATH}" \ -pyarch "${TEST_DATA_DIR}/venv.zip" \ -pyexec "venv.zip/.conda/bin/python" \ - -c org.apache.flink.python.tests.BlinkBatchPythonUdfSqlJob \ + -c org.apache.flink.python.tests.BatchPythonUdfSqlJob \ "${FLINK_PYTHON_TEST_DIR}/target/PythonUdfSqlJobExample.jar" echo "Test using python udf in sql client:\n" diff --git a/flink-end-to-end-tests/test-scripts/test_tpch.sh b/flink-end-to-end-tests/test-scripts/test_tpch.sh index 6ed6264bdcd92..cea0a8e09c927 100755 --- a/flink-end-to-end-tests/test-scripts/test_tpch.sh +++ b/flink-end-to-end-tests/test-scripts/test_tpch.sh @@ -68,7 +68,6 @@ do cat > "$INIT_SQL" << EOF ${SOURCES_SQL} ${SINK_SQL} -SET table.planner=blink; SET execution.runtime-mode=batch; SET parallelism.default=2; EOF diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java index 518b067a94cf6..fd398e53e7050 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java @@ -105,10 +105,10 @@ public void testSerDe() throws Exception { .toInstant(); Map map = new HashMap<>(); - map.put("flink", 123L); + map.put("element", 123L); Map multiSet = new HashMap<>(); - multiSet.put("blink", 2); + multiSet.put("element", 2); Map> nestedMap = new HashMap<>(); Map innerMap = new HashMap<>(); @@ -135,8 +135,8 @@ public void testSerDe() throws Exception { root.put("timestamp3", "1990-10-14T12:12:43.123"); root.put("timestamp9", "1990-10-14T12:12:43.123456789"); root.put("timestampWithLocalZone", "1990-10-14T12:12:43.123456789Z"); - root.putObject("map").put("flink", 123); - root.putObject("multiSet").put("blink", 2); + root.putObject("map").put("element", 123); + root.putObject("multiSet").put("element", 2); root.putObject("map2map").putObject("inner_map").put("key", 234); byte[] serializedJson = objectMapper.writeValueAsBytes(root); diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index 5ecb855ac16ee..f4ebe6cd490e8 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -36,8 +36,7 @@ class TableConfig(object): with detailed inline documentation. For more advanced configuration, users can directly access the underlying key-value map via - :func:`~pyflink.table.TableConfig.get_configuration`. Currently, key-value options are only - supported for the Blink planner. + :func:`~pyflink.table.TableConfig.get_configuration`. .. note:: diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 83a92421b9930..ede8332d27aa3 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -1637,8 +1637,7 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t of the TableEnvironment. :param table_config: The configuration of the TableEnvironment, optional. :param environment_settings: The environment settings used to instantiate the - TableEnvironment. It provides the interfaces about planner - selection(flink or blink), optional. + TableEnvironment. :return: The StreamTableEnvironment created from given StreamExecutionEnvironment and configuration. """ diff --git a/flink-python/pyflink/table/tests/test_calc.py b/flink-python/pyflink/table/tests/test_calc.py index f09b1fa6406d4..07e046a2c5e93 100644 --- a/flink-python/pyflink/table/tests/test_calc.py +++ b/flink-python/pyflink/table/tests/test_calc.py @@ -25,10 +25,10 @@ from pyflink.table.expressions import row from pyflink.table.tests.test_types import PythonOnlyPoint, PythonOnlyUDT from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class StreamTableCalcTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableCalcTests(PyFlinkStreamTableTestCase): def test_select(self): t = self.t_env.from_elements([(1, 'hi', 'hello')], ['a', 'b', 'c']) diff --git a/flink-python/pyflink/table/tests/test_column_operation.py b/flink-python/pyflink/table/tests/test_column_operation.py index d0c043e97e89b..edfc50051d843 100644 --- a/flink-python/pyflink/table/tests/test_column_operation.py +++ b/flink-python/pyflink/table/tests/test_column_operation.py @@ -16,10 +16,10 @@ # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class StreamTableColumnsOperationTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableColumnsOperationTests(PyFlinkStreamTableTestCase): def test_add_columns(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) diff --git a/flink-python/pyflink/table/tests/test_correlate.py b/flink-python/pyflink/table/tests/test_correlate.py index 4f3af5a2a239d..cebd7d193a460 100644 --- a/flink-python/pyflink/table/tests/test_correlate.py +++ b/flink-python/pyflink/table/tests/test_correlate.py @@ -16,10 +16,10 @@ # limitations under the License. ################################################################################ from pyflink.table import expressions as expr -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class CorrelateTests(PyFlinkBlinkStreamTableTestCase): +class CorrelateTests(PyFlinkStreamTableTestCase): def test_join_lateral(self): t_env = self.t_env diff --git a/flink-python/pyflink/table/tests/test_dependency.py b/flink-python/pyflink/table/tests/test_dependency.py index 7567b933c7584..2b9fa66a9da1c 100644 --- a/flink-python/pyflink/table/tests/test_dependency.py +++ b/flink-python/pyflink/table/tests/test_dependency.py @@ -26,8 +26,8 @@ from pyflink.table import expressions as expr from pyflink.table.udf import udf from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import (PyFlinkBlinkStreamTableTestCase, - PyFlinkBlinkBatchTableTestCase) +from pyflink.testing.test_case_utils import (PyFlinkStreamTableTestCase, + PyFlinkBatchTableTestCase) class DependencyTests(object): @@ -65,12 +65,12 @@ def plus_two(i): self.assert_equals(actual, ["+I[3, 1]", "+I[4, 2]", "+I[5, 3]"]) -class BlinkBatchDependencyTests(DependencyTests, PyFlinkBlinkBatchTableTestCase): +class BatchDependencyTests(DependencyTests, PyFlinkBatchTableTestCase): pass -class BlinkStreamDependencyTests(DependencyTests, PyFlinkBlinkStreamTableTestCase): +class StreamDependencyTests(DependencyTests, PyFlinkStreamTableTestCase): def test_set_requirements_without_cached_directory(self): requirements_txt_path = os.path.join(self.tempdir, str(uuid.uuid4())) diff --git a/flink-python/pyflink/table/tests/test_distinct.py b/flink-python/pyflink/table/tests/test_distinct.py index 894aea70b8904..3fcac71a3ff96 100644 --- a/flink-python/pyflink/table/tests/test_distinct.py +++ b/flink-python/pyflink/table/tests/test_distinct.py @@ -16,10 +16,10 @@ # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class StreamTableDistinctTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableDistinctTests(PyFlinkStreamTableTestCase): def test_distinct(self): t = self.t_env.from_elements([(1, "Hi", "Hello")], ['a', 'b', 'c']) diff --git a/flink-python/pyflink/table/tests/test_explain.py b/flink-python/pyflink/table/tests/test_explain.py index 06a7ce2ea612e..c651fec87f0b2 100644 --- a/flink-python/pyflink/table/tests/test_explain.py +++ b/flink-python/pyflink/table/tests/test_explain.py @@ -16,11 +16,11 @@ # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase from pyflink.table.explain_detail import ExplainDetail -class StreamTableExplainTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableExplainTests(PyFlinkStreamTableTestCase): def test_explain(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) diff --git a/flink-python/pyflink/table/tests/test_expression.py b/flink-python/pyflink/table/tests/test_expression.py index 454a77dee9bce..1e7ef7bab003b 100644 --- a/flink-python/pyflink/table/tests/test_expression.py +++ b/flink-python/pyflink/table/tests/test_expression.py @@ -29,7 +29,7 @@ from pyflink.testing.test_case_utils import PyFlinkTestCase -class PyFlinkBlinkBatchExpressionTests(PyFlinkTestCase): +class PyFlinkBatchExpressionTests(PyFlinkTestCase): def test_expression(self): expr1 = col('a') diff --git a/flink-python/pyflink/table/tests/test_join.py b/flink-python/pyflink/table/tests/test_join.py index 8a27e33b02959..8ee4b9f29c1d6 100644 --- a/flink-python/pyflink/table/tests/test_join.py +++ b/flink-python/pyflink/table/tests/test_join.py @@ -16,10 +16,10 @@ # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class StreamTableJoinTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableJoinTests(PyFlinkStreamTableTestCase): def test_join_without_where(self): t_env = self.t_env diff --git a/flink-python/pyflink/table/tests/test_pandas_conversion.py b/flink-python/pyflink/table/tests/test_pandas_conversion.py index c9c0986a269c2..6ece7b84e3704 100644 --- a/flink-python/pyflink/table/tests/test_pandas_conversion.py +++ b/flink-python/pyflink/table/tests/test_pandas_conversion.py @@ -23,8 +23,8 @@ from pyflink.common import Row from pyflink.table.types import DataTypes from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase, \ - PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase, \ + PyFlinkStreamTableTestCase class PandasConversionTestBase(object): @@ -172,14 +172,14 @@ def assert_equal_field(self, expected_field, result_field): self.assertTrue(expected_field == result_field) -class BlinkBatchPandasConversionTests(PandasConversionTests, - PandasConversionITTests, - PyFlinkBlinkBatchTableTestCase): +class BatchPandasConversionTests(PandasConversionTests, + PandasConversionITTests, + PyFlinkBatchTableTestCase): pass -class BlinkStreamPandasConversionTests(PandasConversionITTests, - PyFlinkBlinkStreamTableTestCase): +class StreamPandasConversionTests(PandasConversionITTests, + PyFlinkStreamTableTestCase): def test_to_pandas_with_event_time(self): self.t_env.get_config().get_configuration().set_string("parallelism.default", "1") # create source file path diff --git a/flink-python/pyflink/table/tests/test_pandas_udaf.py b/flink-python/pyflink/table/tests/test_pandas_udaf.py index 97944fd99016b..ecf2ae290a729 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udaf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udaf.py @@ -21,11 +21,11 @@ from pyflink.table.types import DataTypes from pyflink.table.udf import udaf, udf, AggregateFunction from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase, \ - PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase, \ + PyFlinkStreamTableTestCase -class BatchPandasUDAFITTests(PyFlinkBlinkBatchTableTestCase): +class BatchPandasUDAFITTests(PyFlinkBatchTableTestCase): def test_check_result_type(self): def pandas_udaf(): @@ -280,7 +280,7 @@ def test_over_window_aggregate_function(self): "+I[3, 2.0, 3, 2.0, 1.0, 1.0, 2.0, 2.0, 1.0, 1.0]"]) -class StreamPandasUDAFITTests(PyFlinkBlinkStreamTableTestCase): +class StreamPandasUDAFITTests(PyFlinkStreamTableTestCase): def test_sliding_group_window_over_time(self): # create source file path import tempfile diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 77061ec638462..1d838dffdd729 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -24,8 +24,8 @@ from pyflink.table.tests.test_udf import SubtractOne from pyflink.table.udf import udf from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase, \ - PyFlinkBlinkStreamTableTestCase, PyFlinkTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase, \ + PyFlinkStreamTableTestCase, PyFlinkTestCase class PandasUDFTests(PyFlinkTestCase): @@ -306,7 +306,7 @@ def result_type_not_series(i): with self.assertRaisesRegex(Py4JJavaError, expected_regex=msg): t.select(result_type_not_series(t.a)).to_pandas() - def test_data_types_only_supported_in_blink_planner(self): + def test_data_types(self): import pandas as pd timezone = self.t_env.get_config().get_local_timezone() @@ -338,13 +338,13 @@ def local_zoned_timestamp_func(local_zoned_timestamp_param): self.assert_equals(actual, ["+I[1970-01-02T00:00:00.123Z]"]) -class BlinkBatchPandasUDFITTests(PandasUDFITTests, - PyFlinkBlinkBatchTableTestCase): +class BatchPandasUDFITTests(PandasUDFITTests, + PyFlinkBatchTableTestCase): pass -class BlinkStreamPandasUDFITTests(PandasUDFITTests, - PyFlinkBlinkStreamTableTestCase): +class StreamPandasUDFITTests(PandasUDFITTests, + PyFlinkStreamTableTestCase): pass diff --git a/flink-python/pyflink/table/tests/test_row_based_operation.py b/flink-python/pyflink/table/tests/test_row_based_operation.py index b6091706cc7b4..61387f511ab6d 100644 --- a/flink-python/pyflink/table/tests/test_row_based_operation.py +++ b/flink-python/pyflink/table/tests/test_row_based_operation.py @@ -22,8 +22,8 @@ from pyflink.table.types import DataTypes from pyflink.table.udf import udf, udtf, udaf, AggregateFunction, TableAggregateFunction, udtaf from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase, \ - PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase, \ + PyFlinkStreamTableTestCase class RowBasedOperationTests(object): @@ -137,7 +137,7 @@ def split(x): "+I[1, 5, 1, 5, 1, 5]", "+I[1, 6, 1, 6, 1, 6]", "+I[1, 7, 1, 7, 1, 7]"]) -class BatchRowBasedOperationITTests(RowBasedOperationTests, PyFlinkBlinkBatchTableTestCase): +class BatchRowBasedOperationITTests(RowBasedOperationTests, PyFlinkBatchTableTestCase): def test_aggregate_with_pandas_udaf(self): t = self.t_env.from_elements( [(1, 2, 3), (2, 1, 3), (1, 5, 4), (1, 8, 6), (2, 3, 4)], @@ -237,7 +237,7 @@ def test_window_aggregate_with_pandas_udaf(self): "+I[2018-03-11 04:59:59.999, 8.0, 8]"]) -class StreamRowBasedOperationITTests(RowBasedOperationTests, PyFlinkBlinkStreamTableTestCase): +class StreamRowBasedOperationITTests(RowBasedOperationTests, PyFlinkStreamTableTestCase): def test_aggregate(self): import pandas as pd t = self.t_env.from_elements( diff --git a/flink-python/pyflink/table/tests/test_schema_operation.py b/flink-python/pyflink/table/tests/test_schema_operation.py index ecd7d899a7c60..3df9dd34b96d5 100644 --- a/flink-python/pyflink/table/tests/test_schema_operation.py +++ b/flink-python/pyflink/table/tests/test_schema_operation.py @@ -17,10 +17,10 @@ ################################################################################ from pyflink.table.table_schema import TableSchema from pyflink.table.types import DataTypes -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase -class StreamTableSchemaTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableSchemaTests(PyFlinkStreamTableTestCase): def test_print_schema(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) diff --git a/flink-python/pyflink/table/tests/test_set_operation.py b/flink-python/pyflink/table/tests/test_set_operation.py index c25ec59a15eb5..7a56eb59fd947 100644 --- a/flink-python/pyflink/table/tests/test_set_operation.py +++ b/flink-python/pyflink/table/tests/test_set_operation.py @@ -16,10 +16,10 @@ # # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase -class StreamTableSetOperationTests(PyFlinkBlinkBatchTableTestCase): +class StreamTableSetOperationTests(PyFlinkBatchTableTestCase): data1 = [(1, "Hi", "Hello")] data2 = [(3, "Hello", "Hello")] diff --git a/flink-python/pyflink/table/tests/test_sort.py b/flink-python/pyflink/table/tests/test_sort.py index 3e78689008864..5c25fbf7b15b6 100644 --- a/flink-python/pyflink/table/tests/test_sort.py +++ b/flink-python/pyflink/table/tests/test_sort.py @@ -16,10 +16,10 @@ # limitations under the License. ################################################################################ -from pyflink.testing.test_case_utils import PyFlinkBlinkBatchTableTestCase +from pyflink.testing.test_case_utils import PyFlinkBatchTableTestCase -class BatchTableSortTests(PyFlinkBlinkBatchTableTestCase): +class BatchTableSortTests(PyFlinkBatchTableTestCase): def test_order_by_offset_fetch(self): t = self.t_env.from_elements([(1, "Hello")], ["a", "b"]) diff --git a/flink-python/pyflink/table/tests/test_sql.py b/flink-python/pyflink/table/tests/test_sql.py index f391a280a5ab7..dbaa6244aef7e 100644 --- a/flink-python/pyflink/table/tests/test_sql.py +++ b/flink-python/pyflink/table/tests/test_sql.py @@ -23,11 +23,11 @@ from pyflink.java_gateway import get_gateway from pyflink.table import DataTypes, ResultKind from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase, \ +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ PyFlinkTestCase -class StreamSqlTests(PyFlinkBlinkStreamTableTestCase): +class StreamSqlTests(PyFlinkStreamTableTestCase): def test_sql_ddl(self): self.t_env.execute_sql("create temporary function func1 as " diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index 7ec3bbc488bfc..b4aec45c42f33 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -34,7 +34,7 @@ from pyflink.table.types import RowType, Row from pyflink.testing import source_sink_utils from pyflink.testing.test_case_utils import \ - PyFlinkBlinkBatchTableTestCase, PyFlinkBlinkStreamTableTestCase, \ + PyFlinkBatchTableTestCase, PyFlinkStreamTableTestCase, \ _load_specific_flink_module_jars from pyflink.util.java_utils import get_j_env_configuration @@ -258,7 +258,7 @@ def test_to_retract_stream(self): self.assertEqual(result, expected) -class BlinkStreamTableEnvironmentTests(TableEnvironmentTest, PyFlinkBlinkStreamTableTestCase): +class StreamTableEnvironmentTests(TableEnvironmentTest, PyFlinkStreamTableTestCase): def test_collect_with_retract(self): expected_row_kinds = [RowKind.INSERT, RowKind.UPDATE_BEFORE, RowKind.UPDATE_AFTER, @@ -329,7 +329,7 @@ def test_collect_for_all_data_types(self): self.assertEqual(expected_result, collected_result) -class BlinkBatchTableEnvironmentTests(PyFlinkBlinkBatchTableTestCase): +class BatchTableEnvironmentTests(PyFlinkBatchTableTestCase): def test_explain_with_multi_sinks(self): t_env = self.t_env diff --git a/flink-python/pyflink/table/tests/test_types.py b/flink-python/pyflink/table/tests/test_types.py index 0311d5ad7465f..7520fd68a7a32 100644 --- a/flink-python/pyflink/table/tests/test_types.py +++ b/flink-python/pyflink/table/tests/test_types.py @@ -846,12 +846,12 @@ def test_atomic_type_with_data_type_with_parameters(self): # Legacy type tests Types = gateway.jvm.org.apache.flink.table.api.Types - BlinkBigDecimalTypeInfo = \ + InternalBigDecimalTypeInfo = \ gateway.jvm.org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo java_types = [Types.STRING(), Types.DECIMAL(), - BlinkBigDecimalTypeInfo(12, 5)] + InternalBigDecimalTypeInfo(12, 5)] converted_python_types = [_from_java_type(item) for item in java_types] diff --git a/flink-python/pyflink/table/tests/test_udaf.py b/flink-python/pyflink/table/tests/test_udaf.py index 9bc5ee8650b3d..612f4331971a1 100644 --- a/flink-python/pyflink/table/tests/test_udaf.py +++ b/flink-python/pyflink/table/tests/test_udaf.py @@ -29,7 +29,7 @@ from pyflink.table.expressions import col from pyflink.table.udf import AggregateFunction, udaf from pyflink.table.window import Tumble, Slide, Session -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase class CountAggregateFunction(AggregateFunction): @@ -239,7 +239,7 @@ def get_result_type(self): DataTypes.FIELD("f3", DataTypes.BIGINT())]) -class StreamTableAggregateTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableAggregateTests(PyFlinkStreamTableTestCase): def test_double_aggregate(self): self.t_env.register_function("my_count", CountAggregateFunction()) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index c357554495c2f..a8851180b4081 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -24,8 +24,8 @@ from pyflink.table import DataTypes, expressions as expr from pyflink.table.udf import ScalarFunction, udf from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase, \ - PyFlinkBlinkBatchTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ + PyFlinkBatchTableTestCase class UserDefinedFunctionTests(object): @@ -638,8 +638,8 @@ def float_equal(a, b, rel_tol=1e-09, abs_tol=0.0): return abs(a - b) <= max(rel_tol * max(abs(a), abs(b)), abs_tol) -class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, - PyFlinkBlinkStreamTableTestCase): +class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, + PyFlinkStreamTableTestCase): def test_deterministic(self): add_one = udf(lambda i: i + 1, result_type=DataTypes.BIGINT()) self.assertTrue(add_one._deterministic) @@ -704,7 +704,7 @@ def eval(self, col): self.t_env.create_temporary_system_function( "non-callable-udf", udf(Plus(), DataTypes.BIGINT(), DataTypes.BIGINT())) - def test_data_types_only_supported_in_blink_planner(self): + def test_data_types(self): timezone = self.t_env.get_config().get_local_timezone() local_datetime = pytz.timezone(timezone).localize( datetime.datetime(1970, 1, 1, 0, 0, 0, 123000)) @@ -778,8 +778,8 @@ def test_execute_from_json_plan(self): self.assertEqual(lines, ['1,2', '2,3', '3,4']) -class PyFlinkBlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, - PyFlinkBlinkBatchTableTestCase): +class PyFlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, + PyFlinkBatchTableTestCase): pass diff --git a/flink-python/pyflink/table/tests/test_udtf.py b/flink-python/pyflink/table/tests/test_udtf.py index 79d66ad52209e..a069d908b3d2b 100644 --- a/flink-python/pyflink/table/tests/test_udtf.py +++ b/flink-python/pyflink/table/tests/test_udtf.py @@ -20,8 +20,8 @@ from pyflink.table import DataTypes from pyflink.table.udf import TableFunction, udtf, ScalarFunction, udf from pyflink.testing import source_sink_utils -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase, \ - PyFlinkBlinkBatchTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ + PyFlinkBatchTableTestCase class UserDefinedTableFunctionTests(object): @@ -71,8 +71,8 @@ def _get_output(self, t): return source_sink_utils.results() -class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedTableFunctionTests, - PyFlinkBlinkStreamTableTestCase): +class PyFlinkStreamUserDefinedFunctionTests(UserDefinedTableFunctionTests, + PyFlinkStreamTableTestCase): def test_execute_from_json_plan(self): # create source file path tmp_dir = self.tempdir @@ -124,8 +124,8 @@ def test_execute_from_json_plan(self): self.assertEqual(lines, ['1,1,0', '2,2,0', '3,3,0', '3,3,1']) -class PyFlinkBlinkBatchUserDefinedFunctionTests(UserDefinedTableFunctionTests, - PyFlinkBlinkBatchTableTestCase): +class PyFlinkBatchUserDefinedFunctionTests(UserDefinedTableFunctionTests, + PyFlinkBatchTableTestCase): pass diff --git a/flink-python/pyflink/table/tests/test_window.py b/flink-python/pyflink/table/tests/test_window.py index 283b8641fcb00..0446e6fb598a9 100644 --- a/flink-python/pyflink/table/tests/test_window.py +++ b/flink-python/pyflink/table/tests/test_window.py @@ -20,11 +20,11 @@ from pyflink.table import expressions as expr from pyflink.table.window import Session, Slide, Tumble, Over -from pyflink.testing.test_case_utils import PyFlinkBlinkStreamTableTestCase, \ - PyFlinkBlinkBatchTableTestCase +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ + PyFlinkBatchTableTestCase -class StreamTableWindowTests(PyFlinkBlinkStreamTableTestCase): +class StreamTableWindowTests(PyFlinkStreamTableTestCase): def test_over_window(self): t_env = self.t_env @@ -42,7 +42,7 @@ def test_over_window(self): result.select, "b.sum over w") -class BatchTableWindowTests(PyFlinkBlinkBatchTableTestCase): +class BatchTableWindowTests(PyFlinkBatchTableTestCase): def test_tumble_window(self): t = self.t_env.from_elements([(1, 1, "Hello")], ["a", "b", "c"]) diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index 05ce897713b69..d54dc5265ba8f 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -2667,8 +2667,7 @@ def TIMESTAMP_WITH_LOCAL_TIME_ZONE(precision: int = 6, nullable: bool = True) \ It must have a value between 0 and 9 (both inclusive). (default: 6) :param nullable: boolean, whether the type can be null (None) or not. - .. note:: `LocalZonedTimestampType` is currently only supported in blink planner and the - precision must be 3. + .. note:: `LocalZonedTimestampType` only supports precision of 3 currently. """ return LocalZonedTimestampType(precision, nullable) diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index 5f0c2ae152179..6777f6088a831 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -143,26 +143,26 @@ def prepare_csv_source(cls, path, data, data_types, fields): return CsvTableSource(path, fields, data_types) -class PyFlinkBlinkStreamTableTestCase(PyFlinkTestCase): +class PyFlinkStreamTableTestCase(PyFlinkTestCase): """ - Base class for stream tests of blink planner. + Base class for table stream tests. """ def setUp(self): - super(PyFlinkBlinkStreamTableTestCase, self).setUp() + super(PyFlinkStreamTableTestCase, self).setUp() self.t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) self.t_env.get_config().get_configuration().set_string("parallelism.default", "2") self.t_env.get_config().get_configuration().set_string( "python.fn-execution.bundle.size", "1") -class PyFlinkBlinkBatchTableTestCase(PyFlinkTestCase): +class PyFlinkBatchTableTestCase(PyFlinkTestCase): """ - Base class for batch tests of blink planner. + Base class for table batch tests. """ def setUp(self): - super(PyFlinkBlinkBatchTableTestCase, self).setUp() + super(PyFlinkBatchTableTestCase, self).setUp() self.t_env = TableEnvironment.create(EnvironmentSettings.in_batch_mode()) self.t_env.get_config().get_configuration().set_string("parallelism.default", "2") self.t_env.get_config().get_configuration().set_string( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 1821f5c28c41b..87608aabb0b28 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -205,9 +205,7 @@ private static Field toArrowField(String fieldName, LogicalType logicalType) { return new Field(fieldName, fieldType, children); } - /** - * Creates an {@link ArrowWriter} for blink planner for the specified {@link VectorSchemaRoot}. - */ + /** Creates an {@link ArrowWriter} for the specified {@link VectorSchemaRoot}. */ public static ArrowWriter createRowDataArrowWriter( VectorSchemaRoot root, RowType rowType) { ArrowFieldWriter[] fieldWriters = @@ -346,9 +344,7 @@ private static ArrowFieldWriter createArrowFieldWriterForArray( } } - /** - * Creates an {@link ArrowReader} for blink planner for the specified {@link VectorSchemaRoot}. - */ + /** Creates an {@link ArrowReader} for the specified {@link VectorSchemaRoot}. */ public static RowDataArrowReader createRowDataArrowReader( VectorSchemaRoot root, RowType rowType) { List columnVectors = new ArrayList<>(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java index ef8e3ea8e8707..ebe9d45f0522d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java @@ -172,11 +172,10 @@ public void open() throws Exception { baos = new ByteArrayOutputStreamWithPos(); baosWrapper = new DataOutputViewStreamWrapper(baos); userDefinedFunctionInputType = getUserDefinedFunctionInputType(); - udfInputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionInputType); + udfInputTypeSerializer = PythonTypeUtils.toInternalSerializer(userDefinedFunctionInputType); userDefinedFunctionOutputType = getUserDefinedFunctionOutputType(); udfOutputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionOutputType); + PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); rowDataWrapper = new StreamRecordRowDataWrappingCollector(output); super.open(); } @@ -243,7 +242,7 @@ public PythonEnv getPythonEnv() { @VisibleForTesting TypeSerializer getKeySerializer() { - return PythonTypeUtils.toBlinkTypeSerializer(getKeyType()); + return PythonTypeUtils.toInternalSerializer(getKeyType()); } protected RowType getKeyType() { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperator.java index b32d3550cc638..916e0e8303319 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperator.java @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.typeutils.DataViewUtils; import org.apache.flink.table.types.logical.RowType; -/** The Python AggregateFunction operator for the blink planner. */ +/** The Python AggregateFunction operator. */ @Internal public class PythonStreamGroupAggregateOperator extends AbstractPythonStreamGroupAggregateOperator { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperator.java index 6b2e40e604754..7c76e5a9291ae 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperator.java @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.typeutils.DataViewUtils; import org.apache.flink.table.types.logical.RowType; -/** The Python TableAggregateFunction operator for the blink planner. */ +/** The Python TableAggregateFunction operator. */ @Internal public class PythonStreamGroupTableAggregateOperator extends AbstractPythonStreamGroupAggregateOperator { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java index 5b0e839ff14eb..ff9ad1d8c9a1a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java @@ -65,7 +65,7 @@ import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; -/** The Python Group Window AggregateFunction operator for the blink planner. */ +/** The Python Group Window AggregateFunction operator. */ @Internal public class PythonStreamGroupWindowAggregateOperator extends AbstractPythonStreamAggregateOperator implements Triggerable { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java index ffa0cfa646663..64f0cf4ed41db 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractRowDataPythonScalarFunctionOperator.java @@ -37,7 +37,7 @@ import java.util.Arrays; import java.util.stream.Collectors; -/** The Python {@link ScalarFunction} operator for the blink planner. */ +/** The Python {@link ScalarFunction} operator. */ @Internal public abstract class AbstractRowDataPythonScalarFunctionOperator extends AbstractPythonScalarFunctionOperator { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java index f75808eefe8af..11c9f68d6ba5c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/RowDataPythonScalarFunctionOperator.java @@ -31,7 +31,7 @@ import java.io.IOException; -/** The Python {@link ScalarFunction} operator for the blink planner. */ +/** The Python {@link ScalarFunction} operator. */ @Internal public class RowDataPythonScalarFunctionOperator extends AbstractRowDataPythonScalarFunctionOperator { @@ -66,10 +66,9 @@ public RowDataPythonScalarFunctionOperator( @SuppressWarnings("unchecked") public void open() throws Exception { super.open(); - udfInputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionInputType); + udfInputTypeSerializer = PythonTypeUtils.toInternalSerializer(userDefinedFunctionInputType); udfOutputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionOutputType); + PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java index cc2eca48ca0d9..a63d8fd09f2ac 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/RowDataArrowPythonScalarFunctionOperator.java @@ -30,7 +30,7 @@ import org.apache.flink.table.runtime.operators.python.scalar.AbstractRowDataPythonScalarFunctionOperator; import org.apache.flink.table.types.logical.RowType; -/** Arrow Python {@link ScalarFunction} operator for the blink planner. */ +/** Arrow Python {@link ScalarFunction} operator. */ @Internal public class RowDataArrowPythonScalarFunctionOperator extends AbstractRowDataPythonScalarFunctionOperator { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/RowDataPythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/RowDataPythonTableFunctionOperator.java index 5aff6e50c528d..c6f7fae30c7dd 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/RowDataPythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/RowDataPythonTableFunctionOperator.java @@ -39,7 +39,7 @@ import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.table.types.logical.RowType; -/** The Python {@link TableFunction} operator for the blink planner. */ +/** The Python {@link TableFunction} operator. */ @Internal public class RowDataPythonTableFunctionOperator extends AbstractPythonTableFunctionOperator { @@ -84,9 +84,9 @@ public void open() throws Exception { udtfInputProjection = createUdtfInputProjection(); forwardedInputSerializer = new RowDataSerializer(inputType); udtfInputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionInputType); + PythonTypeUtils.toInternalSerializer(userDefinedFunctionInputType); udtfOutputTypeSerializer = - PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionOutputType); + PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index e537d2b02586e..6a84045fc282e 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -81,8 +81,8 @@ public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { return logicalType.accept(new PythonTypeUtils.LogicalTypeToProtoTypeConverter()); } - public static TypeSerializer toBlinkTypeSerializer(LogicalType logicalType) { - return logicalType.accept(new LogicalTypeToBlinkTypeSerializerConverter()); + public static TypeSerializer toInternalSerializer(LogicalType logicalType) { + return logicalType.accept(new LogicalTypetoInternalSerializerConverter()); } /** @@ -137,7 +137,7 @@ public static long timestampToInternal(java.sql.Timestamp ts) { return time + LOCAL_TZ.getOffset(time); } - private static class LogicalTypeToBlinkTypeSerializerConverter + private static class LogicalTypetoInternalSerializerConverter extends LogicalTypeDefaultVisitor { @Override public TypeSerializer visit(BooleanType booleanType) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/ArrayDataSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/ArrayDataSerializer.java index 7800f01921140..2f0baf31ac703 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/ArrayDataSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/ArrayDataSerializer.java @@ -38,7 +38,7 @@ /** * A {@link TypeSerializer} for {@link ArrayData}. It should be noted that the header will not be * encoded. Currently Python doesn't support BinaryArrayData natively, so we can't use - * BaseArraySerializer in blink directly. + * BaseArraySerializer directly. */ @Internal public class ArrayDataSerializer diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java index 7aa0b3f4d5681..c3664220f05ba 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java @@ -30,9 +30,7 @@ import java.sql.Date; /** - * Takes int instead of long as the serialized value. It not only reduces the length of the - * serialized value, but also makes the serialized value consistent between the legacy planner and - * the blink planner. + * Takes int instead of long as the serialized value. It reduces the length of the serialized value. */ @Internal public class DateSerializer extends TypeSerializerSingleton { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/MapDataSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/MapDataSerializer.java index 9c469a7350e13..dbb31e977f2d4 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/MapDataSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/MapDataSerializer.java @@ -41,7 +41,7 @@ /** * A {@link TypeSerializer} for {@link MapData}. It should be noted that the header will not be * encoded. Currently Python doesn't support BinaryMapData natively, so we can't use - * BaseArraySerializer in blink directly. + * BaseArraySerializer directly. */ @Internal public class MapDataSerializer extends org.apache.flink.table.runtime.typeutils.MapDataSerializer { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/RowDataSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/RowDataSerializer.java index ce25bf49eee83..8e8a0a59f0bf3 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/RowDataSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/RowDataSerializer.java @@ -44,7 +44,7 @@ /** * A {@link TypeSerializer} for {@link RowData}. It should be noted that the row kind will be * encoded as the first 2 bits instead of the first byte. Currently Python doesn't support RowData - * natively, so we can't use RowDataSerializer in blink directly. + * natively, so we can't use RowDataSerializer directly. */ @Internal public class RowDataSerializer extends org.apache.flink.table.runtime.typeutils.RowDataSerializer { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java index 3866bb63bd531..3355b4327ea8f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java @@ -32,7 +32,7 @@ * We create the StringSerializer instead of using the StringSerializer of flink-core module because * the StringSerializer of flink-core module serialize every Char of String in serialize method and * deserialize the Char to build the String. We want to convert String to UTF-8 bytes to serialize - * which is compatible with BinaryStringSerializer in blink. + * which is compatible with BinaryStringSerializer. * *

So we create this StringSerializer (only used in Java and Python data communication in udf). * diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/TimeSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/TimeSerializer.java index 30bea75e2c4b2..84716e3dcdecb 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/TimeSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/TimeSerializer.java @@ -30,9 +30,7 @@ import java.util.TimeZone; /** - * Uses int instead of long as the serialized value. It not only reduces the length of the - * serialized value, but also makes the serialized value consistent between the legacy planner and - * the blink planner. + * Uses int instead of long as the serialized value. It reduces the length of the serialized value. */ @Internal public class TimeSerializer extends TypeSerializerSingleton

For more advanced configuration, users can directly access the underlying key-value map via - * {@link #getConfiguration()}. Currently, key-value options are only supported for the Blink - * planner. Users can configure also underlying execution parameters via this object. E.g. + * {@link #getConfiguration()}. Users can configure also underlying execution parameters via this + * object. + * + *

For example: * *

{@code
  * tEnv.getConfig().addConfiguration(
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 cb0299fc3801f..bcada2be7ca4b 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
@@ -33,8 +33,6 @@
 /**
  * This class holds configuration constants used by Flink's table module.
  *
- * 

This is only used for the Blink planner. - * *

NOTE: All option keys in this class must start with "table.exec". */ @PublicEvolving 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 f05d29fb25127..685dd585df3df 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 @@ -27,8 +27,6 @@ /** * This class holds configuration constants used by Flink's table planner module. * - *

This is only used for the Blink planner. - * *

NOTE: All option keys in this class must start with "table.optimizer". */ @PublicEvolving diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StatementSetImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StatementSetImpl.java index 388dad6351c02..226e6360b397a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StatementSetImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/StatementSetImpl.java @@ -111,8 +111,6 @@ public TableResult execute() { * *

The added statements and Tables will NOT be cleared when executing this method. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES: This is an experimental feature now. * * @return the string json representation of an optimized ExecNode plan for the statements and diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java index 7f4a941bcb12a..63b4f5bc9fe60 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentInternal.java @@ -119,8 +119,6 @@ public interface TableEnvironmentInternal extends TableEnvironment { * given statement. An ExecNode plan can be serialized to json plan, and a json plan can be * deserialized to an ExecNode plan. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES: This is an experimental feature now. * * @param stmt The SQL statement to generate json plan. @@ -133,8 +131,6 @@ public interface TableEnvironmentInternal extends TableEnvironment { * Get the json plan for the given {@link ModifyOperation}s. see {@link #getJsonPlan(String)} * for more info about json plan. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES: This is an experimental feature now. * * @param operations the {@link ModifyOperation}s to generate json plan. @@ -148,8 +144,6 @@ public interface TableEnvironmentInternal extends TableEnvironment { * Returns the execution plan for the given json plan. A SQL statement can be converted to json * plan through {@link #getJsonPlan(String)}. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES: This is an experimental feature now. * * @param jsonPlan The json plan to be explained. @@ -164,8 +158,6 @@ public interface TableEnvironmentInternal extends TableEnvironment { * Execute the given json plan, and return the execution result. A SQL statement can be * converted to json plan through {@link #getJsonPlan(String)}. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES: This is an experimental feature now. * * @param jsonPlan The json plan to be executed. diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java index 5dfd45be2acef..dd1c451e95b82 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java @@ -93,8 +93,6 @@ public interface Planner { * given statement. An ExecNode plan can be serialized to json plan, and a json plan can be * deserialized to an ExecNode plan. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES:: This is an experimental feature now. * * @param modifyOperations the {@link ModifyOperation}s to generate json plan. @@ -107,8 +105,6 @@ public interface Planner { /** * Returns the execution plan for the given json plan. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES:: This is an experimental feature now. * * @param jsonPlan The json plan to be explained. @@ -126,8 +122,6 @@ public interface Planner { * given statement. An ExecNode plan can be serialized to json plan, and a json plan can be * deserialized to an ExecNode plan. * - *

NOTES: Only the Blink planner supports this method. - * *

NOTES:: This is an experimental feature now. * * @param jsonPlan The json plan to be translated. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/OverwritableTableSink.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/OverwritableTableSink.java index 228058db2db21..81619138fe2c8 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/OverwritableTableSink.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/OverwritableTableSink.java @@ -28,8 +28,7 @@ * * @see PartitionableTableSink for the definition of partition. * @deprecated This interface will not be supported in the new sink design around {@link - * DynamicTableSink} which only works with the Blink planner. Use {@link SupportsOverwrite} - * instead. See FLIP-95 for more information. + * DynamicTableSink}. Use {@link SupportsOverwrite} instead. See FLIP-95 for more information. */ @Deprecated @Experimental diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java index e244b2ab0a79f..31ccb8fc2a390 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java @@ -63,8 +63,8 @@ * from each record. * * @deprecated This interface will not be supported in the new sink design around {@link - * DynamicTableSink} which only works with the Blink planner. Use {@link SupportsPartitioning} - * instead. See FLIP-95 for more information. + * DynamicTableSink}. Use {@link SupportsPartitioning} instead. See FLIP-95 for more + * information. */ @Deprecated @Experimental diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/TableSink.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/TableSink.java index e40f0ba903468..87b1906aa46aa 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/TableSink.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/TableSink.java @@ -35,8 +35,7 @@ * * @param The return type of the {@link TableSink}. * @deprecated This interface has been replaced by {@link DynamicTableSink}. The new interface - * consumes internal data structures and only works with the Blink planner. See FLIP-95 for more - * information. + * consumes internal data structures. See FLIP-95 for more information. */ @PublicEvolving public interface TableSink { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedFieldMapping.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedFieldMapping.java index dd944aa97a3a2..a5e739668fd27 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedFieldMapping.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedFieldMapping.java @@ -42,8 +42,7 @@ *

If a mapping is provided, all fields must be explicitly mapped. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. See FLIP-95 for more - * information. + * DynamicTableSource}. See FLIP-95 for more information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedProctimeAttribute.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedProctimeAttribute.java index ea8f4dddfd5ef..8a8f3af34eb27 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedProctimeAttribute.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedProctimeAttribute.java @@ -29,8 +29,8 @@ * Extends a {@link TableSource} to specify a processing time attribute. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use the concept of computed - * columns instead. See FLIP-95 for more information. + * DynamicTableSource}. Use the concept of computed columns instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedRowtimeAttributes.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedRowtimeAttributes.java index cec8775da5c32..cefee8d3f436f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedRowtimeAttributes.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/DefinedRowtimeAttributes.java @@ -30,8 +30,8 @@ * RowtimeAttributeDescriptor}. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use the concept of computed - * columns instead. See FLIP-95 for more information. + * DynamicTableSource}. Use the concept of computed columns instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FieldComputer.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FieldComputer.java index aa894c0a93f63..a884348d6354a 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FieldComputer.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FieldComputer.java @@ -30,8 +30,8 @@ * * @param The result type of the provided expression. * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use the concept of computed - * columns instead. See FLIP-95 for more information. + * DynamicTableSource}. Use the concept of computed columns instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FilterableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FilterableTableSource.java index d3104dd70f99f..ef6fe0cd9ad30 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FilterableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/FilterableTableSource.java @@ -29,8 +29,8 @@ * this interface is able to filter records before returning. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link - * SupportsFilterPushDown} instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link SupportsFilterPushDown} instead. See FLIP-95 for more + * information. */ @Deprecated public interface FilterableTableSource { @@ -40,11 +40,6 @@ public interface FilterableTableSource { * been translated in conjunctive form, and table source can only pick those predicates that it * supports. * - *

WARNING: Flink planner will push down PlannerExpressions (which are - * defined in flink-table-planner module), while Blink planner will push down {@link - * Expression}s. So the implementation for Flink planner and Blink planner should be different - * and incompatible. PlannerExpression will be removed in the future. - * *

After trying to push predicates down, we should return a new {@link TableSource} instance * which holds all pushed down predicates. Even if we actually pushed nothing down, it is * recommended that we still return a new {@link TableSource} instance since we will mark the diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java index 0ce27cc7b02e2..f3d88b24cc40b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java @@ -30,8 +30,8 @@ * but does not need to guarantee that the number must be less than or equal to the limit. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link - * SupportsLimitPushDown} instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link SupportsLimitPushDown} instead. See FLIP-95 for more + * information. */ @Deprecated @Experimental diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LookupableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LookupableTableSource.java index b9d9e60148d8b..d3451a35e7122 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LookupableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LookupableTableSource.java @@ -31,8 +31,7 @@ * * @param type of the result * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link LookupTableSource} - * instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link LookupTableSource} instead. See FLIP-95 for more information. */ @Deprecated @Experimental diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.java index f75dc97a51b7f..ad97beaf71ef0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/NestedFieldsProjectableTableSource.java @@ -30,8 +30,8 @@ * * @param The return type of the {@link TableSource}. * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link - * SupportsProjectionPushDown} instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link SupportsProjectionPushDown} instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java index f2d19c5e6f401..eb24236564668 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java @@ -37,8 +37,8 @@ * should be obtained via partition keys of catalog table. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link - * SupportsPartitionPushDown} instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link SupportsPartitionPushDown} instead. See FLIP-95 for more + * information. */ @Deprecated @Experimental diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/ProjectableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/ProjectableTableSource.java index e33337942fef6..dc7fbf54e171d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/ProjectableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/ProjectableTableSource.java @@ -30,8 +30,8 @@ * * @param The return type of the {@link TableSource}. * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use {@link - * SupportsProjectionPushDown} instead. See FLIP-95 for more information. + * DynamicTableSource}. Use {@link SupportsProjectionPushDown} instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/RowtimeAttributeDescriptor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/RowtimeAttributeDescriptor.java index 5f8cfe10c31c3..8145657a5c7a0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/RowtimeAttributeDescriptor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/RowtimeAttributeDescriptor.java @@ -28,8 +28,8 @@ * Describes a rowtime attribute of a {@link TableSource}. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use the concept of computed - * columns instead. See FLIP-95 for more information. + * DynamicTableSource}. Use the concept of computed columns instead. See FLIP-95 for more + * information. */ @Deprecated public final class RowtimeAttributeDescriptor { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/TableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/TableSource.java index 93a3cced5fa5c..df782902ed07b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/TableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/TableSource.java @@ -42,8 +42,7 @@ * * @param The return type of the {@link TableSource}. * @deprecated This interface has been replaced by {@link DynamicTableSource}. The new interface - * produces internal data structures and only works with the Blink planner. See FLIP-95 for more - * information. + * produces internal data structures. See FLIP-95 for more information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/tsextractors/TimestampExtractor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/tsextractors/TimestampExtractor.java index bba74527694b6..af32aeacfdb57 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/tsextractors/TimestampExtractor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/tsextractors/TimestampExtractor.java @@ -35,8 +35,8 @@ * Provides an expression to extract the timestamp for a rowtime attribute. * * @deprecated This interface will not be supported in the new source design around {@link - * DynamicTableSource} which only works with the Blink planner. Use the concept of computed - * columns instead. See FLIP-95 for more information. + * DynamicTableSource}. Use the concept of computed columns instead. See FLIP-95 for more + * information. */ @Deprecated @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LegacyTypeInfoDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LegacyTypeInfoDataTypeConverter.java index 3011ba62e8ef4..00adca9b1d24d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LegacyTypeInfoDataTypeConverter.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LegacyTypeInfoDataTypeConverter.java @@ -90,7 +90,7 @@ * * @deprecated Use {@link DataTypeFactory#createDataType(TypeInformation)} instead. Note that this * method will not create legacy types anymore. It fully uses the new type system available only - * in the Blink planner. + * in the planner. */ @Internal @Deprecated diff --git a/flink-table/flink-table-planner-blink/pom.xml b/flink-table/flink-table-planner-blink/pom.xml index c6737b353a506..08468c196f0d8 100644 --- a/flink-table/flink-table-planner-blink/pom.xml +++ b/flink-table/flink-table-planner-blink/pom.xml @@ -28,13 +28,12 @@ under the License. flink-table-planner-blink_${scala.binary.version} - Flink : Table : Planner Blink + Flink : Table : Planner - This module bridges Table/SQL API and runtime. It contains - all resources that are required during pre-flight and runtime - phase. The content of this module is work-in-progress. It will - replace flink-table-planner once it is stable. See FLINK-11439 - and FLIP-32 for more details. + This module connects Table/SQL API and runtime. It is responsible + for translating and optimizing a table program into a Flink pipeline. + The module can access all resources that are required during + pre-flight and runtime phase for planning. jar diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CustomizedConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CustomizedConvertRule.java index b4d9e57189223..8e1e8529ff43c 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CustomizedConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CustomizedConvertRule.java @@ -99,7 +99,7 @@ public class CustomizedConvertRule implements CallExpressionConvertRule { DEFINITION_RULE_MAP.put( BuiltInFunctionDefinitions.SQRT, CustomizedConvertRule::convertSqrt); - // blink expression + // planner specific expression DEFINITION_RULE_MAP.put( InternalFunctionDefinitions.THROW_EXCEPTION, CustomizedConvertRule::convertThrowException); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/OverConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/OverConvertRule.java index 62df20db22ea6..34586763251f4 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/OverConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/OverConvertRule.java @@ -170,7 +170,7 @@ private RexNode createCollation( default: if (nullDirection == null) { // Set the null direction if not specified. - // Consistent with HIVE/SPARK/MYSQL/BLINK-RUNTIME. + // Consistent with HIVE/SPARK/MYSQL if (FlinkPlannerImpl.defaultNullCollation() .last(direction.equals(RelFieldCollation.Direction.DESCENDING))) { kinds.add(SqlKind.NULLS_LAST); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/RichTableSourceQueryOperation.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/RichTableSourceQueryOperation.java index ff5fdf3e9debb..ed6aabd6bc143 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/RichTableSourceQueryOperation.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/RichTableSourceQueryOperation.java @@ -48,7 +48,7 @@ public RichTableSourceQueryOperation( super(tableSource, false); Preconditions.checkArgument( tableSource instanceof StreamTableSource, - "Blink planner should always use StreamTableSource."); + "Planner should always use StreamTableSource."); this.statistic = statistic; this.identifier = identifier; } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java index 85ee364bccdb9..44656abe0ca02 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java @@ -174,7 +174,7 @@ private static FlinkPreparingTableBase convertSourceTable( if (!(tableSource instanceof StreamTableSource || tableSource instanceof LookupableTableSource)) { throw new ValidationException( - "Only StreamTableSource and LookupableTableSource can be used in Blink planner."); + "Only StreamTableSource and LookupableTableSource can be used in planner."); } if (!isStreamingMode && tableSource instanceof StreamTableSource diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/InternalConfigOptions.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/InternalConfigOptions.java index f9efc7cfaa11c..868ce0381366a 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/InternalConfigOptions.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/InternalConfigOptions.java @@ -26,8 +26,6 @@ /** * This class holds internal configuration constants used by Flink's table module. * - *

This is only used for the Blink planner. - * *

NOTE: All option keys in this class must start with "__" and end up with "__", and all options * shouldn't expose to users, all options should erase after plan finished. */ diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 18ce788f955e6..0f584fa89567c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -581,7 +581,7 @@ object FlinkTypeFactory { throw new TableException( s"TIME precision is not supported: ${relDataType.getPrecision}") } - // blink runner support precision 3, but for consistent with flink runner, we set to 0. + // the planner supports precision 3, but for consistency with old planner, we set it to 0. new TimeType() case TIMESTAMP => new TimestampType(relDataType.getPrecision) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index c16ab9e824b22..195b99b71a7d4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -66,7 +66,7 @@ import java.util.TimeZone import _root_.scala.collection.JavaConversions._ /** - * Implementation of [[Planner]] for blink planner. It supports only streaming use cases. + * Implementation of a [[Planner]]. It supports only streaming use cases. * (The new [[org.apache.flink.table.sources.InputFormatTableSource]] should work, but will be * handled as streaming sources, and no batch specific optimizations will be applied). * diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/parse/SetOperationParseStrategyTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/parse/SetOperationParseStrategyTest.java index 9703f9775e8bb..3269281d11b53 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/parse/SetOperationParseStrategyTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/parse/SetOperationParseStrategyTest.java @@ -29,12 +29,18 @@ public class SetOperationParseStrategyTest { @Test public void testMatches() { assertTrue(SetOperationParseStrategy.INSTANCE.match("SET")); - assertTrue(SetOperationParseStrategy.INSTANCE.match("SET table.planner = blink")); - assertTrue(SetOperationParseStrategy.INSTANCE.match("SET table.planner = 'blink'")); + assertTrue( + SetOperationParseStrategy.INSTANCE.match( + "SET table.local-time-zone = Europe/Berlin")); + assertTrue( + SetOperationParseStrategy.INSTANCE.match( + "SET table.local-time-zone = 'Europe/Berlin'")); } @Test public void testDoesNotMatchQuotedKey() { - assertFalse(SetOperationParseStrategy.INSTANCE.match("SET 'table.planner' = blink")); + assertFalse( + SetOperationParseStrategy.INSTANCE.match( + "SET 'table.local-time-zone' = Europe/Berlin")); } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala index 409bdf3cbbab1..960e523645aa0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala @@ -220,7 +220,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "Func8('a', 'a')", "c") - // TODO fix FLINK-13580 to blink-planner + // TODO fix FLINK-13580 // testAllApis( // Func21('f15), // "Func21(f15)", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala index 29618c7478f9f..2d23a83e29bdc 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala @@ -808,7 +808,7 @@ abstract class AggregateITCaseBase(testName: String) extends BatchTestBase { ) } - // NOTE: select from values -- supported by Spark, but not Blink + // NOTE: select from values -- supported by Spark, but not us // "select sum(a) over () from values 1.0, 2.0, 3.0 T(a)" @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala index 8aed33bc9a18f..8db0b977a6995 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala @@ -678,7 +678,7 @@ class AggregateITCase( tEnv.sqlQuery(sqlQuery).toRetractStream[Row].addSink(sink) env.execute() - // TODO: [BLINK-16716210] the string result of collect is not determinist + // TODO: the string result of collect is not deterministic // TODO: sort the map result in the future val expected = List( "1,{1=1}", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala index 2b75d59179e32..9d73e5dd62d02 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala @@ -43,7 +43,7 @@ class CorrelateITCase extends StreamingTestBase { } @Test - // BLINK-13614111: Fix IndexOutOfBoundsException when UDTF is used on the + // Fix IndexOutOfBoundsException when UDTF is used on the // same name field of different tables def testUdtfForSameFieldofDifferentSource(): Unit = { val data = List( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala index 7bb92a40cde93..cfecba2bcb2cc 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala @@ -142,8 +142,8 @@ class MatchRecognizeITCase(backend: StateBackendMode) extends StreamingWithState def testCodeSplitsAreProperlyGenerated(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) - // TODO: this code is ported from flink-planner, - // However code split is not supported in blink-planner yet. + // TODO: this code is ported from old planner, + // However code split is not supported in planner yet. tEnv.getConfig.setMaxGeneratedCodeLength(1) val data = new mutable.MutableList[(Int, String, String, String)] diff --git a/flink-table/flink-table-runtime-blink/pom.xml b/flink-table/flink-table-runtime-blink/pom.xml index ff0ff7fd71873..f14ace3f8883d 100644 --- a/flink-table/flink-table-runtime-blink/pom.xml +++ b/flink-table/flink-table-runtime-blink/pom.xml @@ -28,12 +28,10 @@ under the License. flink-table-runtime-blink_${scala.binary.version} - Flink : Table : Runtime Blink + Flink : Table : Runtime This module contains classes that are required by a task manager for - execution of table programs. The content of this module is work-in-progress. - It will replace the runtime classes contained in flink-table-planner once - it is stable. See FLINK-11439 and FLIP-32 for more details. + execution of table programs. jar diff --git a/flink-table/flink-table-uber-blink/pom.xml b/flink-table/flink-table-uber-blink/pom.xml index 776a5506f8d7a..5cf46eb825a7b 100644 --- a/flink-table/flink-table-uber-blink/pom.xml +++ b/flink-table/flink-table-uber-blink/pom.xml @@ -28,11 +28,10 @@ under the License. flink-table-uber-blink_${scala.binary.version} - Flink : Table : Uber Blink + Flink : Table : Uber This module contains the entire Table/SQL distribution for writing table programs - within the table ecosystem or between other Flink APIs. This module uses the Blink planner - for generating optimized runnable plan from relational query. Users can either use the + within the table ecosystem or between other Flink APIs. Users can either use the Scala or Java programming language. diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 5cda0278fb536..6b9479e74b321 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -92,8 +92,8 @@ jobs: module: python libraries: module: libraries - blink_planner: - module: blink_planner + table: + module: table connectors: module: connectors kafka_gelly: diff --git a/tools/ci/stage.sh b/tools/ci/stage.sh index b8a048bdf208a..f87b7b9bbd775 100755 --- a/tools/ci/stage.sh +++ b/tools/ci/stage.sh @@ -21,7 +21,7 @@ STAGE_COMPILE="compile" STAGE_CORE="core" STAGE_PYTHON="python" STAGE_LIBRARIES="libraries" -STAGE_BLINK_PLANNER="blink_planner" +STAGE_TABLE="table" STAGE_CONNECTORS="connectors" STAGE_KAFKA_GELLY="kafka/gelly" STAGE_TESTS="tests" @@ -53,7 +53,9 @@ flink-external-resources/flink-external-resource-gpu" MODULES_LIBRARIES="\ flink-libraries/flink-cep,\ flink-libraries/flink-cep-scala,\ -flink-libraries/flink-state-processing-api,\ +flink-libraries/flink-state-processing-api" + +MODULES_TABLE="\ flink-table/flink-sql-parser,\ flink-table/flink-sql-parser-hive,\ flink-table/flink-table-common,\ @@ -61,9 +63,7 @@ flink-table/flink-table-api-java,\ flink-table/flink-table-api-scala,\ flink-table/flink-table-api-java-bridge,\ flink-table/flink-table-api-scala-bridge,\ -flink-table/flink-sql-client" - -MODULES_BLINK_PLANNER="\ +flink-table/flink-sql-client,\ flink-table/flink-table-planner-blink,\ flink-table/flink-table-runtime-blink" @@ -145,8 +145,8 @@ function get_compile_modules_for_stage() { (${STAGE_LIBRARIES}) echo "-pl $MODULES_LIBRARIES -am" ;; - (${STAGE_BLINK_PLANNER}) - echo "-pl $MODULES_BLINK_PLANNER -am" + (${STAGE_TABLE}) + echo "-pl $MODULES_TABLE -am" ;; (${STAGE_CONNECTORS}) echo "-pl $MODULES_CONNECTORS -am" @@ -177,16 +177,17 @@ function get_test_modules_for_stage() { local modules_core=$MODULES_CORE local modules_libraries=$MODULES_LIBRARIES - local modules_blink_planner=$MODULES_BLINK_PLANNER + local modules_table=$MODULES_TABLE + local modules_kafka_gelly=$MODULES_KAFKA_GELLY local modules_connectors=$MODULES_CONNECTORS local modules_tests=$MODULES_TESTS local negated_core=\!${MODULES_CORE//,/,\!} local negated_libraries=\!${MODULES_LIBRARIES//,/,\!} - local negated_blink_planner=\!${MODULES_BLINK_PLANNER//,/,\!} + local negated_table=\!${MODULES_TABLE//,/,\!} local negated_kafka_gelly=\!${MODULES_KAFKA_GELLY//,/,\!} local negated_connectors=\!${MODULES_CONNECTORS//,/,\!} local negated_tests=\!${MODULES_TESTS//,/,\!} - local modules_misc="$negated_core,$negated_libraries,$negated_blink_planner,$negated_connectors,$negated_kafka_gelly,$negated_tests" + local modules_misc="$negated_core,$negated_libraries,$negated_table,$negated_connectors,$negated_kafka_gelly,$negated_tests" local modules_finegrained_resource_management=$MODULES_FINEGRAINED_RESOURCE_MANAGEMENT case ${stage} in @@ -196,14 +197,14 @@ function get_test_modules_for_stage() { (${STAGE_LIBRARIES}) echo "-pl $modules_libraries" ;; - (${STAGE_BLINK_PLANNER}) - echo "-pl $modules_blink_planner" + (${STAGE_TABLE}) + echo "-pl $modules_table" ;; (${STAGE_CONNECTORS}) echo "-pl $modules_connectors" ;; (${STAGE_KAFKA_GELLY}) - echo "-pl $MODULES_KAFKA_GELLY" + echo "-pl $modules_kafka_gelly" ;; (${STAGE_TESTS}) echo "-pl $modules_tests"